You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2015/10/29 13:22:34 UTC

[01/13] flink git commit: [FLINK-2800] [kryo] Fix Kryo serialization to clear buffered data

Repository: flink
Updated Branches:
  refs/heads/master 2b8303cdc -> 676854b92


[FLINK-2800] [kryo] Fix Kryo serialization to clear buffered data

The Kryo serializer uses Kryo's Output class to buffer individual write operations before
it is written to the underlying output stream. This Output class is flushed by Flink's
KryoSerializer upon finishing its serialize call. However, in case of an exception when
flushing the Output, the buffered data is kept in the buffer. Since Flink uses EOFExceptions
to mark that an underlying buffer is full and has to be spilled, for example, it can happen
that the record triggering the spilling is written twice after it is rewritten. The reason
is that Kryo's Output buffer still contains the serialization data of the failed attempt which
is also flushed to the emptied output stream.

This duplication of records can lead to corrupted data which eventually let's the Flink program
crash. The problem is solved by clearing Kryo's Output when the flush operation was not successful.

This closes #1308


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

Branch: refs/heads/master
Commit: b654e989b60c6607f5f5bf757130c86bd0facd8c
Parents: 2b8303c
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Oct 28 18:40:41 2015 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 10:18:07 2015 +0100

----------------------------------------------------------------------
 .../typeutils/runtime/kryo/KryoSerializer.java  |  17 +-
 .../runtime/kryo/KryoClearedBufferTest.java     | 283 +++++++++++++++++++
 2 files changed, 297 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b654e989/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index f825fc6..8549e26 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -58,7 +58,7 @@ import java.util.Objects;
  * @param <T> The type to be serialized.
  */
 public class KryoSerializer<T> extends TypeSerializer<T> {
-	
+
 	private static final long serialVersionUID = 3L;
 
 	// ------------------------------------------------------------------------
@@ -82,7 +82,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 	
 	private transient Input input;
 	private transient Output output;
-	
+
 	// ------------------------------------------------------------------------
 
 	public KryoSerializer(Class<T> type, ExecutionConfig executionConfig){
@@ -182,11 +182,22 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 			previousOut = target;
 		}
 
+		// Sanity check: Make sure that the output is cleared/has been flushed by the last call
+		// otherwise data might be written multiple times in case of a previous EOFException
+		if (output.position() != 0) {
+			throw new IllegalStateException("The Kryo Output still contains data from a previous " +
+				"serialize call. It has to be flushed or cleared at the end of the serialize call.");
+		}
+
 		try {
 			kryo.writeClassAndObject(output, record);
 			output.flush();
 		}
 		catch (KryoException ke) {
+			// make sure that the Kryo output buffer is cleared in case that we can recover from
+			// the exception (e.g. EOFException which denotes buffer full)
+			output.clear();
+
 			Throwable cause = ke.getCause();
 			if (cause instanceof EOFException) {
 				throw (EOFException) cause;
@@ -212,7 +223,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 		} catch (KryoException ke) {
 			Throwable cause = ke.getCause();
 
-			if(cause instanceof EOFException) {
+			if (cause instanceof EOFException) {
 				throw (EOFException) cause;
 			} else {
 				throw ke;

http://git-wip-us.apache.org/repos/asf/flink/blob/b654e989/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
new file mode 100644
index 0000000..ab2e45f
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java
@@ -0,0 +1,283 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.api.java.typeutils.runtime.kryo;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class KryoClearedBufferTest {
+
+	/**
+	 * Tests that the kryo output buffer is cleared in case of an exception. Flink uses the
+	 * EOFException to signal that a buffer is full. In such a case, the record which was tried
+	 * to be written will be rewritten. Therefore, eventually buffered data of this record has
+	 * to be cleared.
+	 */
+	@Test
+	public void testOutputBufferedBeingClearedInCaseOfException() throws Exception {
+		ExecutionConfig executionConfig = new ExecutionConfig();
+		executionConfig.registerTypeWithKryoSerializer(TestRecord.class, new TestRecordSerializer());
+		executionConfig.registerKryoType(TestRecord.class);
+
+		KryoSerializer<TestRecord> kryoSerializer = new KryoSerializer<TestRecord>(
+			TestRecord.class,
+			executionConfig);
+
+		int size = 94;
+		int bufferSize = 150;
+
+		TestRecord testRecord = new TestRecord(size);
+
+		TestDataOutputView target = new TestDataOutputView(bufferSize);
+
+		kryoSerializer.serialize(testRecord, target);
+
+		try {
+			kryoSerializer.serialize(testRecord, target);
+			Assert.fail("Expected an EOFException.");
+		} catch(EOFException eofException) {
+			// expected exception
+			// now the Kryo Output should have been cleared
+		}
+
+		TestRecord actualRecord = kryoSerializer.deserialize(new ByteArrayInputView(target.getBuffer()));
+
+		Assert.assertEquals(testRecord, actualRecord);
+
+		target.clear();
+
+		// if the kryo output has been cleared then we can serialize our test record into the target
+		// because the target buffer 150 bytes can host one TestRecord (total serialization size 100)
+		kryoSerializer.serialize(testRecord, target);
+
+		byte[] buffer = target.getBuffer();
+		int counter = 0;
+
+		for (int i = 0; i < buffer.length; i++) {
+			if(buffer[i] == 42) {
+				counter++;
+			}
+		}
+
+		Assert.assertEquals(size, counter);
+	}
+
+	public static class TestRecord {
+		private byte[] buffer;
+
+		public TestRecord(int size) {
+			buffer = new byte[size];
+
+			Arrays.fill(buffer, (byte)42);
+		}
+
+		public TestRecord(byte[] buffer){
+			this.buffer = buffer;
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj instanceof TestRecord) {
+				TestRecord record = (TestRecord) obj;
+
+				return Arrays.equals(buffer, record.buffer);
+			} else {
+				return false;
+			}
+		}
+	}
+
+	public static class TestRecordSerializer extends Serializer<TestRecord> implements Serializable {
+
+		private static final long serialVersionUID = 6971996565421454985L;
+
+		@Override
+		public void write(Kryo kryo, Output output, TestRecord object) {
+			output.writeInt(object.buffer.length);
+			output.write(object.buffer);
+		}
+
+		@Override
+		public TestRecord read(Kryo kryo, Input input, Class<TestRecord> type) {
+			int length = input.readInt();
+			byte[] buffer = input.readBytes(length);
+
+			return new TestRecord(buffer);
+		}
+	}
+
+	public static class TestDataOutputView implements DataOutputView {
+
+		private byte[] buffer;
+		private int position;
+
+		public TestDataOutputView(int size) {
+			buffer = new byte[size];
+			position = 0;
+		}
+
+		public void clear() {
+			position = 0;
+		}
+
+		public byte[] getBuffer() {
+			return buffer;
+		}
+
+		public void checkSize(int numBytes) throws EOFException {
+			if (position + numBytes > buffer.length) {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public void skipBytesToWrite(int numBytes) throws IOException {
+			checkSize(numBytes);
+
+			position += numBytes;
+		}
+
+		@Override
+		public void write(DataInputView source, int numBytes) throws IOException {
+			checkSize(numBytes);
+
+			byte[] tempBuffer = new byte[numBytes];
+
+			source.read(tempBuffer);
+
+			System.arraycopy(tempBuffer, 0, buffer, position, numBytes);
+
+			position += numBytes;
+		}
+
+		@Override
+		public void write(int b) throws IOException {
+			checkSize(4);
+
+			position += 4;
+		}
+
+		@Override
+		public void write(byte[] b) throws IOException {
+			checkSize(b.length);
+
+			System.arraycopy(b, 0, buffer, position, b.length);
+			position += b.length;
+		}
+
+		@Override
+		public void write(byte[] b, int off, int len) throws IOException {
+			checkSize(len);
+
+			System.arraycopy(b, off, buffer, position, len);
+
+			position += len;
+		}
+
+		@Override
+		public void writeBoolean(boolean v) throws IOException {
+			checkSize(1);
+			position += 1;
+		}
+
+		@Override
+		public void writeByte(int v) throws IOException {
+			checkSize(1);
+
+			buffer[position] = (byte)v;
+
+			position++;
+		}
+
+		@Override
+		public void writeShort(int v) throws IOException {
+			checkSize(2);
+
+			position += 2;
+		}
+
+		@Override
+		public void writeChar(int v) throws IOException {
+			checkSize(1);
+			position++;
+		}
+
+		@Override
+		public void writeInt(int v) throws IOException {
+			checkSize(4);
+
+			position += 4;
+		}
+
+		@Override
+		public void writeLong(long v) throws IOException {
+			checkSize(8);
+			position += 8;
+		}
+
+		@Override
+		public void writeFloat(float v) throws IOException {
+			checkSize(4);
+			position += 4;
+		}
+
+		@Override
+		public void writeDouble(double v) throws IOException {
+			checkSize(8);
+			position += 8;
+		}
+
+		@Override
+		public void writeBytes(String s) throws IOException {
+			byte[] sBuffer = s.getBytes();
+			checkSize(sBuffer.length);
+			System.arraycopy(sBuffer, 0, buffer, position, sBuffer.length);
+			position += sBuffer.length;
+		}
+
+		@Override
+		public void writeChars(String s) throws IOException {
+			byte[] sBuffer = s.getBytes();
+			checkSize(sBuffer.length);
+			System.arraycopy(sBuffer, 0, buffer, position, sBuffer.length);
+			position += sBuffer.length;
+		}
+
+		@Override
+		public void writeUTF(String s) throws IOException {
+			byte[] sBuffer = s.getBytes();
+			checkSize(sBuffer.length);
+			System.arraycopy(sBuffer, 0, buffer, position, sBuffer.length);
+			position += sBuffer.length;
+		}
+	}
+}


[06/13] flink git commit: [FLINK-2559] Clean up JavaDocs

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 33d5a3c..3722908 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -133,7 +133,8 @@ public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<
 
 	/**
 	 * Turns off chaining for this operator so thread co-location will not be
-	 * used as an optimization. </p> Chaining can be turned off for the whole
+	 * used as an optimization.
+	 * <p> Chaining can be turned off for the whole
 	 * job by {@link StreamExecutionEnvironment#disableOperatorChaining()}
 	 * however it is not advised for performance considerations.
 	 * 
@@ -279,7 +280,8 @@ public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<
 	 * maximum parallelism operator in that group. Task chaining is only
 	 * possible within one resource group. By calling this method, this
 	 * operators starts a new resource group and all subsequent operators will
-	 * be added to this group unless specified otherwise. </p> Please note that
+	 * be added to this group unless specified otherwise.
+	 * <p> Please note that
 	 * local executions have by default as many available task slots as the
 	 * environment parallelism, so in order to start a new resource group the
 	 * degree of parallelism for the operators must be decreased from the
@@ -296,7 +298,8 @@ public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<
 	 * Isolates the operator in its own resource group. This will cause the
 	 * operator to grab as many task slots as its degree of parallelism. If
 	 * there are no free resources available, the job will fail to start. It
-	 * also disables chaining for this operator </p>All subsequent operators are
+	 * also disables chaining for this operator.
+	 * <p>All subsequent operators are
 	 * assigned to the default resource group.
 	 * 
 	 * @return The operator with isolated resource group.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 98506e0..210447d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -183,7 +183,7 @@ public abstract class StreamExecutionEnvironment {
 	 * output buffers. By default the output buffers flush frequently to provide
 	 * low latency and to aid smooth developer experience. Setting the parameter
 	 * can result in three logical modes:
-	 * <p/>
+	 * <p>
 	 * <ul>
 	 * <li>
 	 * A positive integer triggers flushing periodically by that integer</li>
@@ -466,7 +466,7 @@ public abstract class StreamExecutionEnvironment {
 
 	/**
 	 * Adds a new Kryo default serializer to the Runtime.
-	 * <p/>
+	 * <p>
 	 * Note that the serializer instance must be serializable (as defined by
 	 * java.io.Serializable), because it may be distributed to the worker nodes
 	 * by java serialization.
@@ -495,7 +495,7 @@ public abstract class StreamExecutionEnvironment {
 
 	/**
 	 * Registers the given type with a Kryo Serializer.
-	 * <p/>
+	 * <p>
 	 * Note that the serializer instance must be serializable (as defined by
 	 * java.io.Serializable), because it may be distributed to the worker nodes
 	 * by java serialization.
@@ -851,7 +851,7 @@ public abstract class StreamExecutionEnvironment {
 	 * objects,
 	 * rather than Java Strings. StringValues can be used to tune implementations to be less object and garbage
 	 * collection heavy.
-	 * <p/>
+	 * <p>
 	 * The file will be read with the system's default character set.
 	 *
 	 * @param filePath
@@ -873,7 +873,7 @@ public abstract class StreamExecutionEnvironment {
 	 * objects, rather than Java Strings. StringValues can be used to tune implementations to be less object and
 	 * garbage
 	 * collection heavy.
-	 * <p/>
+	 * <p>
 	 * The {@link java.nio.charset.Charset} with the given name will be used to read the files.
 	 *
 	 * @param filePath
@@ -992,7 +992,7 @@ public abstract class StreamExecutionEnvironment {
 	 * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are
 	 * decoded by the system's default character set. On the termination of the socket server connection retries can be
 	 * initiated.
-	 * <p/>
+	 * <p>
 	 * Let us note that the socket itself does not report on abort and as a consequence retries are only initiated when
 	 * the socket was gracefully terminated.
 	 *
@@ -1050,7 +1050,7 @@ public abstract class StreamExecutionEnvironment {
 
 	/**
 	 * Generic method to create an input data stream with {@link org.apache.flink.api.common.io.InputFormat}.
-	 * <p/>
+	 * <p>
 	 * Since all data streams need specific information about their types, this method needs to determine the type of
 	 * the data produced by the input format. It will attempt to determine the data type by reflection, unless the
 	 * input
@@ -1194,7 +1194,7 @@ public abstract class StreamExecutionEnvironment {
 	 * Triggers the program execution. The environment will execute all parts of
 	 * the program that have resulted in a "sink" operation. Sink operations are
 	 * for example printing results or forwarding them to a message queue.
-	 * <p/>
+	 * <p>
 	 * The program execution will be logged and displayed with a generated
 	 * default name.
 	 *
@@ -1209,7 +1209,7 @@ public abstract class StreamExecutionEnvironment {
 	 * Triggers the program execution. The environment will execute all parts of
 	 * the program that have resulted in a "sink" operation. Sink operations are
 	 * for example printing results or forwarding them to a message queue.
-	 * <p/>
+	 * <p>
 	 * The program execution will be logged and displayed with the provided name
 	 *
 	 * @param jobName

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
index dd08b2a..24f1a41 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
@@ -36,7 +36,6 @@ import org.apache.flink.api.common.functions.AbstractRichFunction;
  *     <li>Use {@link org.apache.flink.api.common.functions.RuntimeContext#getIndexOfThisSubtask()} to
  *         determine which subtask the current instance of the function executes.</li>
  * </ul>
- * </p>
  *
  * @param <OUT> The type of the records produced by this source.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
index 886d6e7..0e201e4 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
@@ -43,12 +43,12 @@ import java.io.Serializable;
  * This is the basic pattern one should follow when implementing a (checkpointed) source:
  * </p>
  *
- * {@code
+ * <pre>{@code
  *  public class ExampleSource<T> implements SourceFunction<T>, Checkpointed<Long> {
  *      private long count = 0L;
  *      private volatile boolean isRunning = true;
  *
- *      @Override
+ *      {@literal @}Override
  *      public void run(SourceContext<T> ctx) {
  *          while (isRunning && count < 1000) {
  *              synchronized (ctx.getCheckpointLock()) {
@@ -58,18 +58,18 @@ import java.io.Serializable;
  *          }
  *      }
  *
- *      @Override
+ *      {@literal @}Override
  *      public void cancel() {
  *          isRunning = false;
  *      }
  *
- *      @Override
+ *      {@literal @}Override
  *      public Long snapshotState(long checkpointId, long checkpointTimestamp) { return count; }
  *
- *      @Override
+ *      {@literal @}Override
  *      public void restoreState(Long state) { this.count = state; }
  * }
- * </pre>
+ * }</pre>
  *
  *
  * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
index 4a87eb3..8bd0e48 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
@@ -61,7 +61,7 @@ import java.util.Map;
  * in the graph with the desired property. For example, if you have this graph:
  *
  * <pre>
- *     Map-1 -> HashPartition-2 -> Map-3
+ *     Map-1 -&gt; HashPartition-2 -&gt; Map-3
  * </pre>
  *
  * where the numbers represent transformation IDs. We first recurse all the way down. {@code Map-1}

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
index 4e6dc42..fd48fa7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
@@ -47,7 +47,7 @@ import java.util.Collection;
  * <p>
  * The following graph of {@code StreamTransformations}:
  *
- * <pre>
+ * <pre>{@code
  *   Source              Source        
  *      +                   +           
  *      |                   |           
@@ -72,11 +72,11 @@ import java.util.Collection;
  *                |                     
  *                v                     
  *              Sink 
- * </pre>
+ * }</pre>
  *
  * Would result in this graph of operations at runtime:
  *
- * <pre>
+ * <pre>{@code
  *  Source              Source
  *    +                   +
  *    |                   |
@@ -86,7 +86,7 @@ import java.util.Collection;
  *              |
  *              v
  *             Sink
- * </pre>
+ * }</pre>
  *
  * The information about partitioning, union, split/select end up being encoded in the edges
  * that connect the sources to the map operation.
@@ -233,8 +233,7 @@ public abstract class StreamTransformation<T> {
 	/**
 	 * Returns the buffer timeout of this {@code StreamTransformation}.
 	 *
-	 * <p>
-	 * {@see #setBufferTimeout}
+	 * @see #setBufferTimeout(long)
 	 */
 	public long getBufferTimeout() {
 		return bufferTimeout;
@@ -253,8 +252,7 @@ public abstract class StreamTransformation<T> {
 	/**
 	 * Returns the {@code ResourceStrategy} of this {@code StreamTransformation}.
 	 *
-	 * <p>
-	 * {@see #setResourceStrategy}
+	 * @see #setResourceStrategy(StreamGraph.ResourceStrategy)
 	 */
 	public StreamGraph.ResourceStrategy getResourceStrategy() {
 		return resourceStrategy;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
index 7f9356a..e0e9202 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
@@ -54,8 +54,7 @@ public final class Time extends AbstractTime {
 	 * Creates a new {@link Time} of the given duration and {@link TimeUnit}.
 	 *
 	 * <p>The {@code Time} refers to the time characteristic that is set on the dataflow via
-	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#
-	 * setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}.
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}.
 	 *
 	 * @param size The duration of time.
 	 * @param unit The unit of time of the duration, for example {@code TimeUnit.SECONDS}.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 8c58e29..bae0128 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -63,7 +63,7 @@ import org.slf4j.LoggerFactory;
  * thus have multiple ends.
  *
  * The life cycle of the task is set up as follows: 
- * <pre>
+ * <pre>{@code
  *  -- registerInputOutput()
  *         |
  *         +----> Create basic utils (config, etc) and load the chain of operators
@@ -80,7 +80,7 @@ import org.slf4j.LoggerFactory;
  *        +----> dispose-operators()
  *        +----> common cleanup
  *        +----> task specific cleanup()
- * </pre>
+ * }</pre>
  *
  * <p> The {@code StreamTask} has a lock object called {@code lock}. All calls to methods on a
  * {@code StreamOperator} must be synchronized on this lock object to ensure that no methods

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
index 7fb8ba3..f6e297c 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
@@ -49,7 +49,7 @@ import java.io.IOException;
  *
  * <p>
  * When using this you need to add the following line to your test class to setup Powermock:
- * {@code @PrepareForTest({ResultPartitionWriter.class})}
+ * {@code {@literal @}PrepareForTest({ResultPartitionWriter.class})}
  */
 public class OneInputStreamTaskTestHarness<IN, OUT> extends StreamTaskTestHarness<OUT> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index 6c48668..655608b 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -60,7 +60,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
  *
  * <p>
  * When using this you need to add the following line to your test class to setup Powermock:
- * {@code @PrepareForTest({ResultPartitionWriter.class})}
+ * {@code {@literal @}PrepareForTest({ResultPartitionWriter.class})}
  */
 public class StreamTaskTestHarness<OUT> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
index 2b20101..13a9782 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
@@ -57,7 +57,7 @@ import java.util.List;
  *
  * <p>
  * When using this you need to add the following line to your test class to setup Powermock:
- * {@code @PrepareForTest({ResultPartitionWriter.class})}
+ * {@code {@literal @}PrepareForTest({ResultPartitionWriter.class})}
  */
 public class TwoInputStreamTaskTestHarness<IN1, IN2, OUT> extends StreamTaskTestHarness<OUT> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
index 2ea61d7..27710d7 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
@@ -38,14 +38,14 @@ import java.util.Collection;
  *
  * <pre>{@code
  *
- *   @Test
+ *   {@literal @}Test
  *   public void someTest() {
  *       ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
  *       // test code
  *       env.execute();
  *   }
  *
- *   @Test
+ *   {@literal @}Test
  *   public void anotherTest() {
  *       ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
  *       // test code

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java
index 96eb1fc..2fcc523 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/wordcount/WordCount.java
@@ -45,6 +45,12 @@ import java.util.concurrent.TimeUnit;
 /**
  * Implements a word count which takes the input file and counts the number of
  * the occurrences of each word in the file.
+ *
+ * <br><br>
+ *
+ * <b>Note</b>: This example uses the out-dated Record API.
+ * It is recommended to use the new Java API.
+ *
  */
 @SuppressWarnings("deprecation")
 public class WordCount implements Program, ProgramDescription {


[10/13] flink git commit: [FLINK-2934] Remove placeholder pages for job.statistics, taskmanager.log and taskmanager.stdout

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
index 9d3e171..4134274 100644
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html
@@ -36,7 +36,6 @@ limitations under the License.
 <nav ng-if="job" class="navbar navbar-default navbar-fixed-top navbar-main-additional">
   <ul class="nav nav-tabs">
     <li ui-sref-active="active"><a ui-sref=".plan.overview">Plan</a></li>
-    <li ui-sref-active="active"><a ui-sref=".statistics">Job Accumulators / Statistics</a></li>
     <li ui-sref-active="active"><a ui-sref=".timeline">Timeline</a></li>
     <li ui-sref-active="active"><a ui-sref=".exceptions">Exceptions</a></li>
     <li ui-sref-active="active"><a ui-sref=".properties">Properties</a></li>

http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.html b/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.html
index 17cb834..c89970e 100644
--- a/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.html
+++ b/flink-runtime-web/web-dashboard/web/partials/taskmanager/taskmanager.html
@@ -26,8 +26,6 @@ limitations under the License.
 <nav ng-if="metrics.id" class="navbar navbar-default navbar-fixed-top navbar-main-additional">
   <ul class="nav nav-tabs">
     <li ui-sref-active="active"><a ui-sref=".metrics">Metrics</a></li>
-    <li ui-sref-active="active"><a ui-sref=".log">Logs</a></li>
-    <li ui-sref-active="active"><a ui-sref=".stdout">Stdout</a></li>
   </ul>
 </nav>
 <div id="content-inner" class="has-navbar-main-additional">


[12/13] flink git commit: [FLINK-2934] Remove placeholder pages for job.statistics, taskmanager.log and taskmanager.stdout

Posted by fh...@apache.org.
[FLINK-2934] Remove placeholder pages for job.statistics, taskmanager.log and taskmanager.stdout

This closes #1307


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

Branch: refs/heads/master
Commit: 1fd386e74c14f54a7ffc7d9bf62c96c790aea635
Parents: 680b5a9
Author: Sachin Goel <sa...@gmail.com>
Authored: Wed Oct 28 18:00:54 2015 +0530
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 11:01:51 2015 +0100

----------------------------------------------------------------------
 .../web-dashboard/app/partials/jobs/job.jade    |  3 --
 .../app/partials/jobs/job.statistics.jade       | 35 --------------------
 .../app/partials/taskmanager/taskmanager.jade   |  7 ----
 .../taskmanager/taskmanager.logfile.jade        | 17 ----------
 .../taskmanager/taskmanager.stdout.jade         | 17 ----------
 .../web-dashboard/app/scripts/index.coffee      | 18 ----------
 flink-runtime-web/web-dashboard/web/js/index.js | 23 +------------
 .../web-dashboard/web/partials/jobs/job.html    |  1 -
 .../web/partials/taskmanager/taskmanager.html   |  2 --
 9 files changed, 1 insertion(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade
index 9da922a..d0291a4 100644
--- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade
@@ -44,9 +44,6 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main-additional(ng-if="job")
     li(ui-sref-active='active')
       a(ui-sref=".plan.overview") Plan
 
-    li(ui-sref-active='active')
-      a(ui-sref=".statistics") Job Accumulators / Statistics
-
     //- li(ui-sref-active='active' ng-if="job['end-time'] > -1")
     li(ui-sref-active='active')
       a(ui-sref=".timeline") Timeline

http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/app/partials/jobs/job.statistics.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.statistics.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.statistics.jade
deleted file mode 100644
index 22dac9a..0000000
--- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.statistics.jade
+++ /dev/null
@@ -1,35 +0,0 @@
-//
-  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.
-
-table.table.table-properties
-  thead
-    tr
-      th(colspan="2")
-        | Some statistics
-
-  tbody
-    tr
-      td Operator
-      td 1
-
-    tr
-      td Parallelism
-      td 2
-
-    tr
-      td Subtasks-per-instance
-      td 3

http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
index 6eed3b2..23809de 100644
--- a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
@@ -31,13 +31,6 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main-additional(ng-if="metrics
     li(ui-sref-active='active')
       a(ui-sref=".metrics") Metrics
 
-    li(ui-sref-active='active')
-      a(ui-sref=".log") Logs
-
-    li(ui-sref-active='active')
-      a(ui-sref=".stdout") Stdout
-
-
 #content-inner.has-navbar-main-additional
   div(ui-view="details")
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade
deleted file mode 100644
index dacca9d..0000000
--- a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade
+++ /dev/null
@@ -1,17 +0,0 @@
-//
-  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.
-

http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade
deleted file mode 100644
index dacca9d..0000000
--- a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade
+++ /dev/null
@@ -1,17 +0,0 @@
-//
-  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.
-

http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/app/scripts/index.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/index.coffee b/flink-runtime-web/web-dashboard/app/scripts/index.coffee
index af1aa3a..b90ee34 100644
--- a/flink-runtime-web/web-dashboard/app/scripts/index.coffee
+++ b/flink-runtime-web/web-dashboard/app/scripts/index.coffee
@@ -197,12 +197,6 @@ angular.module('flinkApp', ['ui.router', 'angularMoment'])
         templateUrl: "partials/jobs/job.timeline.vertex.html"
         controller: 'JobTimelineVertexController'
 
-  .state "single-job.statistics",
-    url: "/statistics"
-    views:
-      details:
-        templateUrl: "partials/jobs/job.statistics.html"
-
   .state "single-job.exceptions",
     url: "/exceptions"
     views:
@@ -243,18 +237,6 @@ angular.module('flinkApp', ['ui.router', 'angularMoment'])
       details:
         templateUrl: "partials/taskmanager/taskmanager.metrics.html"
 
-  .state "single-manager.log",
-    url: "/logfile"
-    views:
-      details:
-        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
-
-  .state "single-manager.stdout",
-    url: "/stdout"
-    views:
-      details:
-        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
-
   .state "jobmanager",
       url: "/jobmanager"
       views:


[08/13] flink git commit: [FLINK-2559] Clean up JavaDocs

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
index 9425291..79944ae 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/WebLogAnalysis.java
@@ -35,7 +35,7 @@ import org.apache.flink.examples.java.relational.util.WebLogDataGenerator;
  * This program processes web logs and relational data. 
  * It implements the following relational query:
  *
- * <code><pre>
+ * <pre>{@code
  * SELECT 
  *       r.pageURL, 
  *       r.pageRank, 
@@ -50,13 +50,13 @@ import org.apache.flink.examples.java.relational.util.WebLogDataGenerator;
  *              WHERE v.destUrl = d.url 
  *                    AND v.visitDate < [date]
  *           );
- * </pre></code>
+ * }</pre>
  *
  * <p>
  * Input files are plain text CSV files using the pipe character ('|') as field separator.
  * The tables referenced in the query can be generated using the {@link WebLogDataGenerator} and 
  * have the following schemas
- * <code><pre>
+ * <pre>{@code
  * CREATE TABLE Documents (
  *                url VARCHAR(100) PRIMARY KEY,
  *                contents TEXT );
@@ -76,7 +76,7 @@ import org.apache.flink.examples.java.relational.util.WebLogDataGenerator;
  *                languageCode VARCHAR(6),
  *                searchWord VARCHAR(32),
  *                duration INT );
- * </pre></code>
+ * }</pre>
  * 
  * <p>
  * Usage: <code>WebLogAnalysis &lt;documents path&gt; &lt;ranks path&gt; &lt;visits path&gt; &lt;result path&gt;</code><br>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/PojoExample.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/PojoExample.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/PojoExample.java
index b001d12..0a8595f 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/PojoExample.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/PojoExample.java
@@ -112,7 +112,7 @@ public class PojoExample {
 	/**
 	 * Implements the string tokenizer that splits sentences into words as a user-defined
 	 * FlatMapFunction. The function takes a line (String) and splits it into 
-	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
+	 * multiple pairs in the form of "(word,1)" ({@code Tuple2<String, Integer>}).
 	 */
 	public static final class Tokenizer implements FlatMapFunction<String, Word> {
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
index e6b8418..c677160 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java
@@ -90,7 +90,7 @@ public class WordCount {
 	/**
 	 * Implements the string tokenizer that splits sentences into words as a user-defined
 	 * FlatMapFunction. The function takes a line (String) and splits it into 
-	 * multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
+	 * multiple pairs in the form of "(word,1)" ({@code Tuple2<String, Integer>}).
 	 */
 	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java
index b8bd7c1..200fd0f 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/wordcount/WordCountMeta.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.ProgramDescription;
 import org.apache.flink.examples.java.wordcount.util.WordCountData;
 
 /**
- * Same as {@link WorkCount} but implements {@link ProgramDescription} interface.
+ * Same as {@link WordCount} but implements {@link ProgramDescription} interface.
  * 
  * <p>
  * The input is a plain text file with lines separated by newline characters.
@@ -35,7 +35,7 @@ import org.apache.flink.examples.java.wordcount.util.WordCountData;
  * This example shows:
  * <ul>
  * <li>how to provide additional information (using {@link ProgramDescription} interface}, that can be displayed by
- * Flink clients, ie, {@link bin/flink} and WebClient</li>
+ * Flink clients, ie, bin/flink and WebClient</li>
  * </ul>
  * 
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
index 3101f35..81a747e 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
@@ -92,7 +92,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * A DataSet represents a collection of elements of the same type.<br/>
+ * A DataSet represents a collection of elements of the same type.<br>
  * A DataSet can be transformed into another DataSet by applying a transformation as for example 
  * <ul>
  *   <li>{@link DataSet#map(org.apache.flink.api.common.functions.MapFunction)},</li>
@@ -193,7 +193,7 @@ public abstract class DataSet<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Applies a Map transformation on this DataSet.<br/>
+	 * Applies a Map transformation on this DataSet.<br>
 	 * The transformation calls a {@link org.apache.flink.api.common.functions.MapFunction} for each element of the DataSet.
 	 * Each MapFunction call returns exactly one element.
 	 * 
@@ -244,7 +244,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Applies a FlatMap transformation on a {@link DataSet}.<br/>
+	 * Applies a FlatMap transformation on a {@link DataSet}.<br>
 	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichFlatMapFunction} for each element of the DataSet.
 	 * Each FlatMapFunction call can return any number of elements including none.
 	 * 
@@ -266,7 +266,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Applies a Filter transformation on a {@link DataSet}.<br/>
+	 * Applies a Filter transformation on a {@link DataSet}.<br>
 	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichFilterFunction} for each element of the DataSet
 	 * and retains only those element for which the function returns true. Elements for 
 	 * which the function returns false are filtered. 
@@ -291,9 +291,9 @@ public abstract class DataSet<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Applies a Project transformation on a {@link Tuple} {@link DataSet}.<br/>
-	 * <b>Note: Only Tuple DataSets can be projected using field indexes.</b><br/>
-	 * The transformation projects each Tuple of the DataSet onto a (sub)set of fields.<br/>
+	 * Applies a Project transformation on a {@link Tuple} {@link DataSet}.<br>
+	 * <b>Note: Only Tuple DataSets can be projected using field indexes.</b><br>
+	 * The transformation projects each Tuple of the DataSet onto a (sub)set of fields.<br>
 	 * Additional fields can be added to the projection by calling {@link ProjectOperator#project(int[])}.
 	 *
 	 * <b>Note: With the current implementation, the Project transformation looses type information.</b>
@@ -315,7 +315,7 @@ public abstract class DataSet<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Applies an Aggregate transformation on a non-grouped {@link Tuple} {@link DataSet}.<br/>
+	 * Applies an Aggregate transformation on a non-grouped {@link Tuple} {@link DataSet}.<br>
 	 * <b>Note: Only Tuple DataSets can be aggregated.</b>
 	 * The transformation applies a built-in {@link Aggregations Aggregation} on a specified field 
 	 *   of a Tuple DataSet. Additional aggregation functions can be added to the resulting 
@@ -424,7 +424,7 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Applies a Reduce transformation on a non-grouped {@link DataSet}.<br/>
+	 * Applies a Reduce transformation on a non-grouped {@link DataSet}.<br>
 	 * The transformation consecutively calls a {@link org.apache.flink.api.common.functions.RichReduceFunction}
 	 *   until only a single element remains which is the result of the transformation.
 	 * A ReduceFunction combines two elements into one new element of the same type.
@@ -444,7 +444,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Applies a GroupReduce transformation on a non-grouped {@link DataSet}.<br/>
+	 * Applies a GroupReduce transformation on a non-grouped {@link DataSet}.<br>
 	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} once with the full DataSet.
 	 * The GroupReduceFunction can iterate over all elements of the DataSet and emit any
 	 *   number of output elements including none.
@@ -561,7 +561,7 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Returns a new set containing the first n elements in this {@link DataSet}.<br/>
+	 * Returns a new set containing the first n elements in this {@link DataSet}.<br>
 	 * @param n The desired number of elements.
 	 * @return A ReduceGroupOperator that represents the DataSet containing the elements.
 	*/
@@ -640,7 +640,7 @@ public abstract class DataSet<T> {
 	/**
 	 * Groups a {@link DataSet} using a {@link KeySelector} function. 
 	 * The KeySelector function is called for each element of the DataSet and extracts a single 
-	 *   key value on which the DataSet is grouped. <br/>
+	 *   key value on which the DataSet is grouped. <br>
 	 * This method returns an {@link UnsortedGrouping} on which one of the following grouping transformation 
 	 *   can be applied. 
 	 * <ul>
@@ -666,8 +666,8 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Groups a {@link Tuple} {@link DataSet} using field position keys.<br/> 
-	 * <b>Note: Field position keys only be specified for Tuple DataSets.</b><br/>
+	 * Groups a {@link Tuple} {@link DataSet} using field position keys.<br> 
+	 * <b>Note: Field position keys only be specified for Tuple DataSets.</b><br>
 	 * The field position keys specify the fields of Tuples on which the DataSet is grouped.
 	 * This method returns an {@link UnsortedGrouping} on which one of the following grouping transformation 
 	 *   can be applied. 
@@ -724,10 +724,10 @@ public abstract class DataSet<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Initiates a Join transformation. <br/>
+	 * Initiates a Join transformation. <br>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * 
 	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
@@ -743,10 +743,10 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Initiates a Join transformation. <br/>
+	 * Initiates a Join transformation. <br>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * 
 	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
@@ -764,12 +764,12 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Join transformation. <br/>
+	 * Initiates a Join transformation. <br>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * This method also gives the hint to the optimizer that the second DataSet to join is much
-	 *   smaller than the first one.<br/>
+	 *   smaller than the first one.<br>
 	 * This method returns a {@link JoinOperatorSets} on which
 	 *   {@link JoinOperatorSets#where(String...)} needs to be called to define the join key of the first
 	 *   joining (i.e., this) DataSet.
@@ -785,12 +785,12 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Initiates a Join transformation.<br/>
+	 * Initiates a Join transformation.<br>
 	 * A Join transformation joins the elements of two 
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine 
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * This method also gives the hint to the optimizer that the second DataSet to join is much
-	 *   larger than the first one.<br/>
+	 *   larger than the first one.<br>
 	 * This method returns a {@link JoinOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 *  
@@ -805,10 +805,10 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Left Outer Join transformation.<br/>
+	 * Initiates a Left Outer Join transformation.<br>
 	 * An Outer Join transformation joins two elements of two
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * Elements of the <b>left</b> DataSet (i.e. {@code this}) that do not have a matching
 	 *   element on the other side are joined with {@code null} and emitted to the
 	 *   resulting DataSet.
@@ -824,10 +824,10 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Left Outer Join transformation.<br/>
+	 * Initiates a Left Outer Join transformation.<br>
 	 * An Outer Join transformation joins two elements of two
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * Elements of the <b>left</b> DataSet (i.e. {@code this}) that do not have a matching
 	 *   element on the other side are joined with {@code null} and emitted to the
 	 *   resulting DataSet.
@@ -854,10 +854,10 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Right Outer Join transformation.<br/>
+	 * Initiates a Right Outer Join transformation.<br>
 	 * An Outer Join transformation joins two elements of two
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * Elements of the <b>right</b> DataSet (i.e. {@code other}) that do not have a matching
 	 *   element on {@code this} side are joined with {@code null} and emitted to the
 	 *   resulting DataSet.
@@ -873,10 +873,10 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Right Outer Join transformation.<br/>
+	 * Initiates a Right Outer Join transformation.<br>
 	 * An Outer Join transformation joins two elements of two
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * Elements of the <b>right</b> DataSet (i.e. {@code other}) that do not have a matching
 	 *   element on {@code this} side are joined with {@code null} and emitted to the
 	 *   resulting DataSet.
@@ -902,10 +902,10 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Full Outer Join transformation.<br/>
+	 * Initiates a Full Outer Join transformation.<br>
 	 * An Outer Join transformation joins two elements of two
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * Elements of <b>both</b> DataSets that do not have a matching
 	 *   element on the opposing side are joined with {@code null} and emitted to the
 	 *   resulting DataSet.
@@ -921,10 +921,10 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Initiates a Full Outer Join transformation.<br/>
+	 * Initiates a Full Outer Join transformation.<br>
 	 * An Outer Join transformation joins two elements of two
 	 *   {@link DataSet DataSets} on key equality and provides multiple ways to combine
-	 *   joining elements into one DataSet.<br/>
+	 *   joining elements into one DataSet.<br>
 	 * Elements of <b>both</b> DataSets that do not have a matching
 	 *   element on the opposing side are joined with {@code null} and emitted to the
 	 *   resulting DataSet.
@@ -953,14 +953,14 @@ public abstract class DataSet<T> {
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * Initiates a CoGroup transformation.<br/>
+	 * Initiates a CoGroup transformation.<br>
 	 * A CoGroup transformation combines the elements of
 	 *   two {@link DataSet DataSets} into one DataSet. It groups each DataSet individually on a key and 
 	 *   gives groups of both DataSets with equal keys together into a {@link org.apache.flink.api.common.functions.RichCoGroupFunction}.
 	 *   If a DataSet has a group with no matching key in the other DataSet, the CoGroupFunction
-	 *   is called with an empty group for the non-existing group.<br/>
+	 *   is called with an empty group for the non-existing group.<br>
 	 * The CoGroupFunction can iterate over the elements of both groups and return any number 
-	 *   of elements including none.<br/>
+	 *   of elements including none.<br>
 	 * This method returns a {@link CoGroupOperatorSets} on which one of the {@code where} methods
 	 * can be called to define the join key of the first joining (i.e., this) DataSet.
 	 * 
@@ -981,8 +981,8 @@ public abstract class DataSet<T> {
 
 	/**
 	 * Continues a Join transformation and defines the {@link Tuple} fields of the second join 
-	 * {@link DataSet} that should be used as join keys.<br/>
-	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+	 * {@link DataSet} that should be used as join keys.<br>
+	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br>
 	 * 
 	 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with 
 	 * the element of the first input being the first field of the tuple and the element of the 
@@ -993,7 +993,7 @@ public abstract class DataSet<T> {
 	 */
 	
 	/**
-	 * Initiates a Cross transformation.<br/>
+	 * Initiates a Cross transformation.<br>
 	 * A Cross transformation combines the elements of two 
 	 *   {@link DataSet DataSets} into one DataSet. It builds all pair combinations of elements of 
 	 *   both DataSets, i.e., it builds a Cartesian product.
@@ -1006,7 +1006,7 @@ public abstract class DataSet<T> {
 	 * <p>
 	 * Call {@link org.apache.flink.api.java.operators.CrossOperator.DefaultCross#with(org.apache.flink.api.common.functions.CrossFunction)} to define a
 	 * {@link org.apache.flink.api.common.functions.CrossFunction} which is called for
-	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br/>
+	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br>
 	 * 
 	 * @param other The other DataSet with which this DataSet is crossed. 
 	 * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements.
@@ -1021,7 +1021,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Initiates a Cross transformation.<br/>
+	 * Initiates a Cross transformation.<br>
 	 * A Cross transformation combines the elements of two 
 	 *   {@link DataSet DataSets} into one DataSet. It builds all pair combinations of elements of 
 	 *   both DataSets, i.e., it builds a Cartesian product.
@@ -1036,7 +1036,7 @@ public abstract class DataSet<T> {
 	 * <p>
 	 * Call {@link org.apache.flink.api.java.operators.CrossOperator.DefaultCross#with(org.apache.flink.api.common.functions.CrossFunction)} to define a
 	 * {@link org.apache.flink.api.common.functions.CrossFunction} which is called for
-	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br/>
+	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br>
 	 * 
 	 * @param other The other DataSet with which this DataSet is crossed. 
 	 * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements.
@@ -1051,7 +1051,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Initiates a Cross transformation.<br/>
+	 * Initiates a Cross transformation.<br>
 	 * A Cross transformation combines the elements of two 
 	 *   {@link DataSet DataSets} into one DataSet. It builds all pair combinations of elements of 
 	 *   both DataSets, i.e., it builds a Cartesian product.
@@ -1066,7 +1066,7 @@ public abstract class DataSet<T> {
 	 * <p>
 	 * Call {@link org.apache.flink.api.java.operators.CrossOperator.DefaultCross#with(org.apache.flink.api.common.functions.CrossFunction)} to define a
 	 * {@link org.apache.flink.api.common.functions.CrossFunction} which is called for
-	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br/>
+	 * each pair of crossed elements. The CrossFunction returns a exactly one element for each pair of input elements.<br>
 	 * 
 	 * @param other The other DataSet with which this DataSet is crossed. 
 	 * @return A DefaultCross that returns a Tuple2 for each pair of crossed elements.
@@ -1343,7 +1343,7 @@ public abstract class DataSet<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Writes a DataSet as a text file to the specified location.<br/>
+	 * Writes a DataSet as a text file to the specified location.<br>
 	 * For each element of the DataSet the result of {@link Object#toString()} is written.  
 	 * 
 	 * @param filePath The path pointing to the location the text file is written to.
@@ -1356,7 +1356,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Writes a DataSet as a text file to the specified location.<br/>
+	 * Writes a DataSet as a text file to the specified location.<br>
 	 * For each element of the DataSet the result of {@link Object#toString()} is written.  
 	 * 
 	 * @param filePath The path pointing to the location the text file is written to.
@@ -1372,7 +1372,7 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Writes a DataSet as a text file to the specified location.<br/>
+	 * Writes a DataSet as a text file to the specified location.<br>
 	 * For each element of the DataSet the result of {@link TextFormatter#format(Object)} is written.
 	 *
 	 * @param filePath The path pointing to the location the text file is written to.
@@ -1386,7 +1386,7 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Writes a DataSet as a text file to the specified location.<br/>
+	 * Writes a DataSet as a text file to the specified location.<br>
 	 * For each element of the DataSet the result of {@link TextFormatter#format(Object)} is written.
 	 *
 	 * @param filePath The path pointing to the location the text file is written to.
@@ -1401,10 +1401,10 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location.<br/>
-	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br/>
+	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location.<br>
+	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br>
 	 * For each Tuple field the result of {@link Object#toString()} is written.
-	 * Tuple fields are separated by the default field delimiter {@code "comma" (,)}.<br/>
+	 * Tuple fields are separated by the default field delimiter {@code "comma" (,)}.<br>
 	 * Tuples are are separated by the newline character ({@code \n}).
 	 * 
 	 * @param filePath The path pointing to the location the CSV file is written to.
@@ -1418,10 +1418,10 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location.<br/>
-	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br/>
+	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location.<br>
+	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br>
 	 * For each Tuple field the result of {@link Object#toString()} is written.
-	 * Tuple fields are separated by the default field delimiter {@code "comma" (,)}.<br/>
+	 * Tuple fields are separated by the default field delimiter {@code "comma" (,)}.<br>
 	 * Tuples are are separated by the newline character ({@code \n}).
 	 * 
 	 * @param filePath The path pointing to the location the CSV file is written to.
@@ -1436,8 +1436,8 @@ public abstract class DataSet<T> {
 	}
 	
 	/**
-	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location with the specified field and line delimiters.<br/>
-	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br/>
+	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location with the specified field and line delimiters.<br>
+	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br>
 	 * For each Tuple field the result of {@link Object#toString()} is written.
 	 * 
 	 * @param filePath The path pointing to the location the CSV file is written to.
@@ -1452,9 +1452,9 @@ public abstract class DataSet<T> {
 	}
 
 	/**
-	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location with the specified field and line delimiters.<br/>
-	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br/>
-	 * For each Tuple field the result of {@link Object#toString()} is written.
+	 * Writes a {@link Tuple} DataSet as a CSV file to the specified location with the specified field and line delimiters.<br>
+	 * <b>Note: Only a Tuple DataSet can written as a CSV file.</b><br>
+§	 * For each Tuple field the result of {@link Object#toString()} is written.
 	 * 
 	 * @param filePath The path pointing to the location the CSV file is written to.
 	 * @param rowDelimiter The row delimiter to separate Tuples.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/Utils.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java
index 45a14a3..bf9ca4c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java
@@ -63,7 +63,6 @@ public final class Utils {
 	 * Returns all GenericTypeInfos contained in a composite type.
 	 *
 	 * @param typeInfo
-	 * @return
 	 */
 	public static void getContainedGenericTypes(CompositeType typeInfo, List<GenericTypeInfo<?>> target) {
 		for(int i = 0; i < typeInfo.getArity(); i++) {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
index bfc1bf0..a92560e 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
@@ -33,8 +33,8 @@ import org.apache.flink.api.common.InvalidProgramException;
  * Semantic annotations can help the Flink optimizer to generate more efficient execution plans for Flink programs.
  * For example, a <i>ForwardedFields</i> assertion for a map-type function can be declared as:
  *
- * <pre><blockquote>
- * \@ForwardedFields({"f0; f1->f2"})
+ * <pre>{@code
+ * {@literal @}ForwardedFields({"f0; f1->f2"})
  * public class MyMapper extends MapFunction<Tuple3<String, String, Integer>, Tuple3<String, Integer, Integer>>
  * {
  *     public Tuple3<String, Integer, Integer> map(Tuple3<String, String, Integer> val) {
@@ -42,7 +42,7 @@ import org.apache.flink.api.common.InvalidProgramException;
  *         return new Tuple3<String, Integer, Integer>(val.f0, val.f2, 1);
  *     }
  * }
- * </blockquote></pre>
+ * }</pre>
  *
  * <p>
  * All annotations take Strings with expressions that refer to (nested) value fields of the input and output types of a function.
@@ -84,17 +84,17 @@ public class FunctionAnnotation {
 	 *
 	 * Fields that are forwarded at the same position can be specified by their position.
 	 * The specified position must be valid for the input and output data type and have the same type.
-	 * For example <code>\@ForwardedFields({"f2"})</code> declares that the third field of a Java input tuple is
+	 * For example {@code {@literal @}ForwardedFields({"f2"})} declares that the third field of a Java input tuple is
 	 * copied to the third field of an output tuple.
 	 *
 	 * Fields which are unchanged copied to another position in the output are declared by specifying the
 	 * source field expression in the input and the target field expression in the output.
-	 * <code>\@ForwardedFields({"f0->f2"})</code> denotes that the first field of the Java input tuple is
+	 * {@code {@literal @}ForwardedFields({"f0->f2"})} denotes that the first field of the Java input tuple is
 	 * unchanged copied to the third field of the Java output tuple. When using the wildcard ("*") ensure that
 	 * the number of declared fields and their types in input and output type match.
 	 *
-	 * Multiple forwarded fields can be annotated in one (<code>\@ForwardedFields({"f2; f3->f0; f4"})</code>)
-	 * or separate Strings (<code>\@ForwardedFields({"f2", "f3->f0", "f4"})</code>).
+	 * Multiple forwarded fields can be annotated in one ({@code {@literal @}ForwardedFields({"f2; f3->f0; f4"})})
+	 * or separate Strings ({@code {@literal @}ForwardedFields({"f2", "f3->f0", "f4"})}).
 	 *
 	 * <b>NOTE: The use of the ForwardedFields annotation is optional.
 	 * If used correctly, it can help the Flink optimizer to generate more efficient execution plans.
@@ -119,17 +119,17 @@ public class FunctionAnnotation {
 	 *
 	 * Fields that are forwarded from the first input at the same position in the output can be
 	 * specified by their position. The specified position must be valid for the input and output data type and have the same type.
-	 * For example <code>\@ForwardedFieldsFirst({"f2"})</code> declares that the third field of a Java input tuple at the first input is
+	 * For example {@code {@literal @}ForwardedFieldsFirst({"f2"})} declares that the third field of a Java input tuple at the first input is
 	 * copied to the third field of an output tuple.
 	 *
 	 * Fields which are unchanged copied to another position in the output are declared by specifying the
 	 * source field expression in the input and the target field expression in the output.
-	 * <code>\@ForwardedFieldsFirst({"f0->f2"})</code> denotes that the first field of the Java input tuple at the first input is
+	 * {@code {@literal @}ForwardedFieldsFirst({"f0->f2"})} denotes that the first field of the Java input tuple at the first input is
 	 * unchanged copied to the third field of the Java output tuple. When using the wildcard ("*") ensure that
 	 * the number of declared fields and their types in input and output type match.
 	 *
-	 * Multiple forwarded fields can be annotated in one (<code>\@ForwardedFieldsFirst({"f2; f3->f0; f4"})</code>)
-	 * or separate Strings (<code>\@ForwardedFieldsFirst({"f2", "f3->f0", "f4"})</code>).
+	 * Multiple forwarded fields can be annotated in one ({@code {@literal @}ForwardedFieldsFirst({"f2; f3->f0; f4"})})
+	 * or separate Strings ({@code {@literal @}ForwardedFieldsFirst({"f2", "f3->f0", "f4"})}).
 	 *
 	 * <b>NOTE: The use of the ForwardedFieldsFirst annotation is optional.
 	 * If used correctly, it can help the Flink optimizer to generate more efficient execution plans.
@@ -157,17 +157,17 @@ public class FunctionAnnotation {
 	 *
 	 * Fields that are forwarded from the second input at the same position in the output can be
 	 * specified by their position. The specified position must be valid for the input and output data type and have the same type.
-	 * For example <code>\@ForwardedFieldsSecond({"f2"})</code> declares that the third field of a Java input tuple at the second input is
+	 * For example {@code {@literal @}ForwardedFieldsSecond({"f2"})} declares that the third field of a Java input tuple at the second input is
 	 * copied to the third field of an output tuple.
 	 *
 	 * Fields which are unchanged copied to another position in the output are declared by specifying the
 	 * source field expression in the input and the target field expression in the output.
-	 * <code>\@ForwardedFieldsSecond({"f0->f2"})</code> denotes that the first field of the Java input tuple at the second input is
+	 * {@code {@literal @}ForwardedFieldsSecond({"f0->f2"})} denotes that the first field of the Java input tuple at the second input is
 	 * unchanged copied to the third field of the Java output tuple. When using the wildcard ("*") ensure that
 	 * the number of declared fields and their types in input and output type match.
 	 *
-	 * Multiple forwarded fields can be annotated in one (<code>\@ForwardedFieldsSecond({"f2; f3->f0; f4"})</code>)
-	 * or separate Strings (<code>\@ForwardedFieldsSecond({"f2", "f3->f0", "f4"})</code>).
+	 * Multiple forwarded fields can be annotated in one ({@code {@literal @}ForwardedFieldsSecond({"f2; f3->f0; f4"})})
+	 * or separate Strings ({@code {@literal @}ForwardedFieldsSecond({"f2", "f3->f0", "f4"})}).
 	 *
 	 * <b>NOTE: The use of the ForwardedFieldsSecond annotation is optional.
 	 * If used correctly, it can help the Flink optimizer to generate more efficient execution plans.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormat.java
index 126b629..967f8c4 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormat.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopInputFormat.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapred.JobConf;
 /**
  * Wrapper for using HadoopInputFormats (mapred-variant) with Flink.
  *
- * The IF is returning a Tuple2<K,V>.
+ * The IF is returning a {@code Tuple2<K,V>}.
  *
  * @param <K> Type of the key
  * @param <V> Type of the value.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormat.java
index 9c4c6d7..b406dae 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormat.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapred/HadoopOutputFormat.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.mapred.OutputCommitter;
 /**
  * Wrapper for using HadoopOutputFormats (mapred-variant) with Flink.
  *
- * The IF is returning a Tuple2<K,V>.
+ * The IF is returning a {@code Tuple2<K,V>}.
  *
  * @param <K> Type of the key
  * @param <V> Type of the value.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java b/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java
index 04d9953..56d2a64 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java
@@ -95,7 +95,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 * </b>
 	 *
 	 * @param partitionFields The field positions of the partitioning keys.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsPartitionedBy(int... partitionFields) {
 		return this.splitsPartitionedBy(null, partitionFields);
@@ -112,7 +112,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 *
 	 * @param partitionMethodId An ID for the method that was used to partition the data across splits.
 	 * @param partitionFields The field positions of the partitioning keys.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsPartitionedBy(String partitionMethodId, int... partitionFields) {
 
@@ -142,7 +142,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 * </b>
 	 *
 	 * @param partitionFields The field expressions of the partitioning keys.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsPartitionedBy(String partitionFields) {
 		return this.splitsPartitionedBy(null, partitionFields);
@@ -160,7 +160,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 *
 	 * @param partitionMethodId An ID for the method that was used to partition the data across splits.
 	 * @param partitionFields The field expressions of the partitioning keys.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsPartitionedBy(String partitionMethodId, String partitionFields) {
 
@@ -194,7 +194,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 * </b>
 	 *
 	 * @param groupFields The field positions of the grouping keys.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsGroupedBy(int... groupFields) {
 
@@ -224,7 +224,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 * </b>
 	 *
 	 * @param groupFields The field expressions of the grouping keys.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsGroupedBy(String groupFields) {
 
@@ -257,7 +257,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 *
 	 * @param orderFields The field positions of the grouping keys.
 	 * @param orders The orders of the fields.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsOrderedBy(int[] orderFields, Order[] orders) {
 
@@ -306,7 +306,7 @@ public class SplitDataProperties<T> implements GenericDataSourceBase.SplitDataPr
 	 *
 	 * @param orderFields The field expressions of the grouping key.
 	 * @param orders The orders of the fields.
-	 * @result This SplitDataProperties object.
+	 * @return This SplitDataProperties object.
 	 */
 	public SplitDataProperties<T> splitsOrderedBy(String orderFields, Order[] orders) {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
index 36378b9..328dff9 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java
@@ -420,7 +420,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * Intermediate step of a CoGroup transformation. <br/>
+	 * Intermediate step of a CoGroup transformation. <br>
 	 * To continue the CoGroup transformation, select the grouping key of the first input {@link DataSet} by calling 
 	 * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets#where(int...)} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets#where(KeySelector)}.
 	 *
@@ -442,9 +442,9 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		}
 
 		/**
-		 * Continues a CoGroup transformation. <br/>
-		 * Defines the {@link Tuple} fields of the first co-grouped {@link DataSet} that should be used as grouping keys.<br/>
-		 * <b>Note: Fields can only be selected as grouping keys on Tuple DataSets.</b><br/>
+		 * Continues a CoGroup transformation. <br>
+		 * Defines the {@link Tuple} fields of the first co-grouped {@link DataSet} that should be used as grouping keys.<br>
+		 * <b>Note: Fields can only be selected as grouping keys on Tuple DataSets.</b><br>
 		 *
 		 *
 		 * @param fields The indexes of the Tuple fields of the first co-grouped DataSets that should be used as keys.
@@ -459,7 +459,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		}
 
 		/**
-		 * Continues a CoGroup transformation. <br/>
+		 * Continues a CoGroup transformation. <br>
 		 * Defines the fields of the first co-grouped {@link DataSet} that should be used as grouping keys. Fields
 		 * are the names of member fields of the underlying type of the data set.
 		 *
@@ -476,9 +476,9 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		}
 
 		/**
-		 * Continues a CoGroup transformation and defines a {@link KeySelector} function for the first co-grouped {@link DataSet}.</br>
+		 * Continues a CoGroup transformation and defines a {@link KeySelector} function for the first co-grouped {@link DataSet}.<br>
 		 * The KeySelector function is called for each element of the first DataSet and extracts a single 
-		 * key value on which the DataSet is grouped. </br>
+		 * key value on which the DataSet is grouped. <br>
 		 * 
 		 * @param keyExtractor The KeySelector function which extracts the key values from the DataSet on which it is grouped.
 		 * @return An incomplete CoGroup transformation. 
@@ -495,7 +495,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 		// ----------------------------------------------------------------------------------------
 
 		/**
-		 * Intermediate step of a CoGroup transformation. <br/>
+		 * Intermediate step of a CoGroup transformation. <br>
 		 * To continue the CoGroup transformation, select the grouping key of the second input {@link DataSet} by calling 
 		 * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(int...)} or {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate#equalTo(KeySelector)}.
 		 *
@@ -518,8 +518,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 
 			/**
 			 * Continues a CoGroup transformation and defines the {@link Tuple} fields of the second co-grouped 
-			 * {@link DataSet} that should be used as grouping keys.<br/>
-			 * <b>Note: Fields can only be selected as grouping keys on Tuple DataSets.</b><br/>
+			 * {@link DataSet} that should be used as grouping keys.<br>
+			 * <b>Note: Fields can only be selected as grouping keys on Tuple DataSets.</b><br>
 			 *
 			 *
 			 * @param fields The indexes of the Tuple fields of the second co-grouped DataSet that should be used as keys.
@@ -532,7 +532,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 
 			/**
 			 * Continues a CoGroup transformation and defines the fields of the second co-grouped
-			 * {@link DataSet} that should be used as grouping keys.<br/>
+			 * {@link DataSet} that should be used as grouping keys.<br>
 			 *
 			 *
 			 * @param fields The  fields of the first co-grouped DataSets that should be used as keys.
@@ -544,9 +544,9 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			}
 
 			/**
-			 * Continues a CoGroup transformation and defines a {@link KeySelector} function for the second co-grouped {@link DataSet}.</br>
+			 * Continues a CoGroup transformation and defines a {@link KeySelector} function for the second co-grouped {@link DataSet}.<br>
 			 * The KeySelector function is called for each element of the second DataSet and extracts a single 
-			 * key value on which the DataSet is grouped. </br>
+			 * key value on which the DataSet is grouped. <br>
 			 * 
 			 * @param keyExtractor The KeySelector function which extracts the key values from the second DataSet on which it is grouped.
 			 * @return An incomplete CoGroup transformation. 
@@ -558,7 +558,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 			}
 
 			/**
-			 * Intermediate step of a CoGroup transformation. <br/>
+			 * Intermediate step of a CoGroup transformation. <br>
 			 * To continue the CoGroup transformation, provide a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} by calling
 			 * {@link org.apache.flink.api.java.operators.CoGroupOperator.CoGroupOperatorSets.CoGroupOperatorSetsPredicate.CoGroupOperatorWithoutFunction#with(org.apache.flink.api.common.functions.CoGroupFunction)}.
 			 *
@@ -634,7 +634,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				}
 
 				/**
-				 * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} to groups of elements with identical keys.<br/>
+				 * Finalizes a CoGroup transformation by applying a {@link org.apache.flink.api.common.functions.RichCoGroupFunction} to groups of elements with identical keys.<br>
 				 * Each CoGroupFunction call returns an arbitrary number of keys. 
 				 * 
 				 * @param function The CoGroupFunction that is called for all groups of elements with identical keys.
@@ -660,8 +660,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				
 				/**
 				 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the first input on the
-				 * specified field in the specified {@link Order}.</br>
-				 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br/>
+				 * specified field in the specified {@link Order}.<br>
+				 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br>
 				 * Groups can be sorted by multiple fields by chaining {@link #sortFirstGroup(int, Order)} calls.
 				 * 
 				 * @param field The Tuple field on which the group is sorted.
@@ -690,8 +690,8 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				
 				/**
 				 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the second input on the
-				 * specified field in the specified {@link Order}.</br>
-				 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br/>
+				 * specified field in the specified {@link Order}.<br>
+				 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br>
 				 * Groups can be sorted by multiple fields by chaining {@link #sortSecondGroup(int, Order)} calls.
 				 * 
 				 * @param field The Tuple field on which the group is sorted.
@@ -720,7 +720,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				
 				/**
 				 * Sorts Pojo or {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the first input on the
-				 * specified field in the specified {@link Order}.</br>
+				 * specified field in the specified {@link Order}.<br>
 				 * Groups can be sorted by multiple fields by chaining {@link #sortFirstGroup(String, Order)} calls.
 				 * 
 				 * @param fieldExpression The expression to the field on which the group is to be sorted.
@@ -745,7 +745,7 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
 				
 				/**
 				 * Sorts Pojo or {@link org.apache.flink.api.java.tuple.Tuple} elements within a group in the second input on the
-				 * specified field in the specified {@link Order}.</br>
+				 * specified field in the specified {@link Order}.<br>
 				 * Groups can be sorted by multiple fields by chaining {@link #sortSecondGroup(String, Order)} calls.
 				 * 
 				 * @param fieldExpression The expression to the field on which the group is to be sorted.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
index ae990ce..17cd423 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
@@ -108,7 +108,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 	// --------------------------------------------------------------------------------------------
 
 	/**
-	 * A Cross transformation that wraps pairs of crossed elements into {@link Tuple2}.<br/>
+	 * A Cross transformation that wraps pairs of crossed elements into {@link Tuple2}.<br>
 	 * It also represents the {@link DataSet} that is the result of a Cross transformation. 
 	 * 
 	 * @param <I1> The type of the first input DataSet of the Cross transformation.
@@ -137,7 +137,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Finalizes a Cross transformation by applying a {@link CrossFunction} to each pair of crossed elements.<br/>
+		 * Finalizes a Cross transformation by applying a {@link CrossFunction} to each pair of crossed elements.<br>
 		 * Each CrossFunction call returns exactly one element. 
 		 * 
 		 * @param function The CrossFunction that is called for each pair of crossed elements.
@@ -157,9 +157,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 		
 		/**
-		 * Initiates a ProjectCross transformation and projects the first cross input<br/>
+		 * Initiates a ProjectCross transformation and projects the first cross input<br>
 		 * If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the first cross input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the first cross input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and
@@ -182,9 +182,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 		
 		/**
-		 * Initiates a ProjectCross transformation and projects the second cross input<br/>
+		 * Initiates a ProjectCross transformation and projects the second cross input<br>
 		 * If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the second cross input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the second cross input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and
@@ -210,7 +210,7 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 
 	/**
 	 * A Cross transformation that projects crossing elements or fields of crossing {@link Tuple Tuples}
-	 * into result {@link Tuple Tuples}. <br/>
+	 * into result {@link Tuple Tuples}. <br>
 	 * It also represents the {@link DataSet} that is the result of a Cross transformation.
 	 *
 	 * @param <I1> The type of the first input DataSet of the Cross transformation.
@@ -250,9 +250,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 		
 		/**
-		 * Continues a ProjectCross transformation and adds fields of the first cross input to the projection.<br/>
+		 * Continues a ProjectCross transformation and adds fields of the first cross input to the projection.<br>
 		 * If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the first cross input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the first cross input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Additional fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and
@@ -277,9 +277,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Continues a ProjectCross transformation and adds fields of the second cross input to the projection.<br/>
+		 * Continues a ProjectCross transformation and adds fields of the second cross input to the projection.<br>
 		 * If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the second cross input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the second cross input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Additional fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.CrossOperator.ProjectCross#projectFirst(int...)} and
@@ -493,9 +493,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Continues a ProjectCross transformation and adds fields of the first cross input.<br/>
+		 * Continues a ProjectCross transformation and adds fields of the first cross input.<br>
 		 * If the first cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the first cross input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the first cross input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#projectFirst(int...)} and
@@ -559,9 +559,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
 		}
 
 		/**
-		 * Continues a ProjectCross transformation and adds fields of the second cross input.<br/>
+		 * Continues a ProjectCross transformation and adds fields of the second cross input.<br>
 		 * If the second cross input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the second cross input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the second cross input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#projectFirst(int...)} and

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
index 5b5b031..48209cf 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java
@@ -95,8 +95,8 @@ public class DataSink<T> {
 
 	/**
 	 * Sorts each local partition of a {@link org.apache.flink.api.java.tuple.Tuple} data set
-	 * on the specified field in the specified {@link Order} before it is emitted by the output format.</br>
-	 * <b>Note: Only tuple data sets can be sorted using integer field indices.</b><br/>
+	 * on the specified field in the specified {@link Order} before it is emitted by the output format.<br>
+	 * <b>Note: Only tuple data sets can be sorted using integer field indices.</b><br>
 	 * The tuple data set can be sorted on multiple fields in different orders
 	 * by chaining {@link #sortLocalOutput(int, Order)} calls.
 	 *
@@ -149,9 +149,9 @@ public class DataSink<T> {
 
 	/**
 	 * Sorts each local partition of a data set on the field(s) specified by the field expression
-	 * in the specified {@link Order} before it is emitted by the output format.</br>
+	 * in the specified {@link Order} before it is emitted by the output format.<br>
 	 * <b>Note: Non-composite types can only be sorted on the full element which is specified by
-	 * a wildcard expression ("*" or "_").</b><br/>
+	 * a wildcard expression ("*" or "_").</b><br>
 	 * Data sets of composite types (Tuple or Pojo) can be sorted on multiple fields in different orders
 	 * by chaining {@link #sortLocalOutput(String, Order)} calls.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
index 3c0d07f..59811b6 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.functions.Partitioner;
 import org.apache.flink.api.java.DataSet;
 
 /**
- * Grouping is an intermediate step for a transformation on a grouped DataSet.<br/>
+ * Grouping is an intermediate step for a transformation on a grouped DataSet.<br>
  * The following transformation can be applied on Grouping:
  * <ul>
  * 	<li>{@link UnsortedGrouping#reduce(org.apache.flink.api.common.functions.ReduceFunction)},</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
index 3679b87..99a1957 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java
@@ -184,7 +184,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * A Join transformation that applies a {@link JoinFunction} on each pair of joining elements.<br/>
+	 * A Join transformation that applies a {@link JoinFunction} on each pair of joining elements.<br>
 	 * It also represents the {@link DataSet} that is the result of a Join transformation. 
 	 * 
 	 * @param <I1> The type of the first input DataSet of the Join transformation.
@@ -525,7 +525,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 	}
 	
 	/**
-	 * A Join transformation that wraps pairs of joining elements into {@link Tuple2}.<br/>
+	 * A Join transformation that wraps pairs of joining elements into {@link Tuple2}.<br>
 	 * It also represents the {@link DataSet} that is the result of a Join transformation. 
 	 * 
 	 * @param <I1> The type of the first input DataSet of the Join transformation.
@@ -545,7 +545,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 
 		/**
-		 * Finalizes a Join transformation by applying a {@link org.apache.flink.api.common.functions.RichFlatJoinFunction} to each pair of joined elements.<br/>
+		 * Finalizes a Join transformation by applying a {@link org.apache.flink.api.common.functions.RichFlatJoinFunction} to each pair of joined elements.<br>
 		 * Each JoinFunction call returns exactly one element. 
 		 * 
 		 * @param function The JoinFunction that is called for each pair of joined elements.
@@ -587,9 +587,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 
 		/**
-		 * Applies a ProjectJoin transformation and projects the first join input<br/>
+		 * Applies a ProjectJoin transformation and projects the first join input<br>
 		 * If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the first join input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the first join input is not a Tuple DataSet, no parameters should be passed.<br>
 		 * 
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
@@ -613,9 +613,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 		
 		/**
-		 * Applies a ProjectJoin transformation and projects the second join input<br/>
+		 * Applies a ProjectJoin transformation and projects the second join input<br>
 		 * If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the second join input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the second join input is not a Tuple DataSet, no parameters should be passed.<br>
 		 * 
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
@@ -657,7 +657,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 	
 	/**
 	 * A Join transformation that projects joining elements or fields of joining {@link Tuple Tuples} 
-	 * into result {@link Tuple Tuples}. <br/>
+	 * into result {@link Tuple Tuples}. <br>
 	 * It also represents the {@link DataSet} that is the result of a Join transformation. 
 	 * 
 	 * @param <I1> The type of the first input DataSet of the Join transformation.
@@ -693,9 +693,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 
 		/**
-		 * Continues a ProjectJoin transformation and adds fields of the first join input to the projection.<br/>
+		 * Continues a ProjectJoin transformation and adds fields of the first join input to the projection.<br>
 		 * If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the first join input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the first join input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Additional fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
@@ -720,9 +720,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 
 		/**
-		 * Continues a ProjectJoin transformation and adds fields of the second join input to the projection.<br/>
+		 * Continues a ProjectJoin transformation and adds fields of the second join input to the projection.<br>
 		 * If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the second join input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the second join input is not a Tuple DataSet, no parameters should be passed.<br>
 		 *
 		 * Additional fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.ProjectJoin#projectFirst(int...)} and
@@ -846,7 +846,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 //	}
 
 	/**
-	 * Intermediate step of a Join transformation. <br/>
+	 * Intermediate step of a Join transformation. <br>
 	 * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
 	 * {@link JoinOperatorSets#where(int...)} or
 	 * {@link JoinOperatorSets#where(org.apache.flink.api.java.functions.KeySelector)}.
@@ -906,7 +906,7 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 
 
 		/**
-		 * Intermediate step of a Join transformation. <br/>
+		 * Intermediate step of a Join transformation. <br>
 		 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(int...)} or
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets.JoinOperatorSetsPredicate#equalTo(KeySelector)}.
@@ -919,9 +919,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 
 			/**
 			 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
-			 * {@link DataSet} that should be used as join keys.<br/>
-			 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
-			 * <p/>
+			 * {@link DataSet} that should be used as join keys.<br>
+			 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br>
+			 * <p>
 			 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
 			 * the element of the first input being the first field of the tuple and the element of the
 			 * second input being the second field of the tuple.
@@ -936,8 +936,8 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 
 			/**
 			 * Continues a Join transformation and defines the fields of the second join
-			 * {@link DataSet} that should be used as join keys.<br/>
-			 * <p/>
+			 * {@link DataSet} that should be used as join keys.<br>
+			 * <p>
 			 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
 			 * the element of the first input being the first field of the tuple and the element of the
 			 * second input being the second field of the tuple.
@@ -951,10 +951,10 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 			}
 
 			/**
-			 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
+			 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.<br>
 			 * The KeySelector function is called for each element of the second DataSet and extracts a single
-			 * key value on which the DataSet is joined. </br>
-			 * <p/>
+			 * key value on which the DataSet is joined. <br>
+			 * <p>
 			 * The resulting {@link DefaultJoin} wraps each pair of joining elements into a {@link Tuple2}, with
 			 * the element of the first input being the first field of the tuple and the element of the
 			 * second input being the second field of the tuple.
@@ -1143,9 +1143,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 		
 		/**
-		 * Continues a ProjectJoin transformation and adds fields of the first join input.<br/>
+		 * Continues a ProjectJoin transformation and adds fields of the first join input.<br>
 		 * If the first join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the first join input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the first join input is not a Tuple DataSet, no parameters should be passed.<br>
 		 * 
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectFirst(int...)} and 
@@ -1207,9 +1207,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
 		}
 		
 		/**
-		 * Continues a ProjectJoin transformation and adds fields of the second join input.<br/>
+		 * Continues a ProjectJoin transformation and adds fields of the second join input.<br>
 		 * If the second join input is a {@link Tuple} {@link DataSet}, fields can be selected by their index.
-		 * If the second join input is not a Tuple DataSet, no parameters should be passed.<br/>
+		 * If the second join input is not a Tuple DataSet, no parameters should be passed.<br>
 		 * 
 		 * Fields of the first and second input can be added by chaining the method calls of
 		 * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#projectFirst(int...)} and

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
index 9301e1a..63fb804 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SingleInputUdfOperator.java
@@ -125,14 +125,14 @@ public abstract class SingleInputUdfOperator<IN, OUT, O extends SingleInputUdfOp
 	 * <p>
 	 * Fields which are unchanged copied to another position in the output are declared by specifying the
 	 * source field reference in the input and the target field reference in the output.
-	 * <code>withForwardedFields("f0->f2")</code> denotes that the first field of the Java input tuple is
+	 * {@code withForwardedFields("f0->f2")} denotes that the first field of the Java input tuple is
 	 * unchanged copied to the third field of the Java output tuple. When using a wildcard ("*") ensure that
 	 * the number of declared fields and their types in input and output type match.
 	 * </p>
 	 *
 	 * <p>
-	 * Multiple forwarded fields can be annotated in one (<code>withForwardedFields("f2; f3->f0; f4")</code>)
-	 * or separate Strings (<code>withForwardedFields("f2", "f3->f0", "f4")</code>).
+	 * Multiple forwarded fields can be annotated in one ({@code withForwardedFields("f2; f3->f0; f4")})
+	 * or separate Strings ({@code withForwardedFields("f2", "f3->f0", "f4")}).
 	 * Please refer to the JavaDoc of {@link org.apache.flink.api.common.functions.Function} or Flink's documentation for
 	 * details on field references such as nested fields and wildcard.
 	 * </p>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
index 4532424..2fa2467 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
@@ -38,7 +38,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
 import com.google.common.base.Preconditions;
 
 /**
- * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.<br/>
+ * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.<br>
  * The following transformation can be applied on sorted groups:
  * <ul>
  * 	<li>{@link SortedGrouping#reduceGroup(org.apache.flink.api.common.functions.GroupReduceFunction)},</li>
@@ -145,7 +145,7 @@ public class SortedGrouping<T> extends Grouping<T> {
 	}
 
 	/**
-	 * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}.<br/>
+	 * Applies a GroupReduce transformation on a grouped and sorted {@link DataSet}.<br>
 	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet.
 	 * A GroupReduceFunction can iterate over all elements of a group and emit any
 	 *   number of output elements including none.
@@ -189,7 +189,7 @@ public class SortedGrouping<T> extends Grouping<T> {
 
 	
 	/**
-	 * Returns a new set containing the first n elements in this grouped and sorted {@link DataSet}.<br/>
+	 * Returns a new set containing the first n elements in this grouped and sorted {@link DataSet}.<br>
 	 * @param n The desired number of elements for each group.
 	 * @return A GroupReduceOperator that represents the DataSet containing the elements.
 	*/
@@ -206,8 +206,8 @@ public class SortedGrouping<T> extends Grouping<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}.</br>
-	 * <b>Note: Only groups of Tuple or Pojo elements can be sorted.</b><br/>
+	 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}.<br>
+	 * <b>Note: Only groups of Tuple or Pojo elements can be sorted.</b><br>
 	 * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls.
 	 * 
 	 * @param field The Tuple field on which the group is sorted.
@@ -235,8 +235,8 @@ public class SortedGrouping<T> extends Grouping<T> {
 	}
 
 	/**
-	 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} or POJO elements within a group on the specified field in the specified {@link Order}.</br>
-	 * <b>Note: Only groups of Tuple or Pojo elements can be sorted.</b><br/>
+	 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} or POJO elements within a group on the specified field in the specified {@link Order}.<br>
+	 * <b>Note: Only groups of Tuple or Pojo elements can be sorted.</b><br>
 	 * Groups can be sorted by multiple fields by chaining {@link #sortGroup(String, Order)} calls.
 	 *
 	 * @param field The Tuple or Pojo field on which the group is sorted.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
index d23dd56..55a8343 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/TwoInputUdfOperator.java
@@ -127,14 +127,14 @@ public abstract class TwoInputUdfOperator<IN1, IN2, OUT, O extends TwoInputUdfOp
 	 * <p>
 	 * Fields which are unchanged copied from the first input to another position in the output are declared
 	 * by specifying the source field reference in the first input and the target field reference in the output.
-	 * <code>withForwardedFieldsFirst("f0->f2")</code> denotes that the first field of the first input Java tuple is
+	 * {@code withForwardedFieldsFirst("f0->f2")} denotes that the first field of the first input Java tuple is
 	 * unchanged copied to the third field of the Java output tuple. When using a wildcard ("*") ensure that
 	 * the number of declared fields and their types in first input and output type match.
 	 * </p>
 	 *
 	 * <p>
-	 * Multiple forwarded fields can be annotated in one (<code>withForwardedFieldsFirst("f2; f3->f0; f4")</code>)
-	 * or separate Strings (<code>withForwardedFieldsFirst("f2", "f3->f0", "f4")</code>).
+	 * Multiple forwarded fields can be annotated in one ({@code withForwardedFieldsFirst("f2; f3->f0; f4")})
+	 * or separate Strings ({@code withForwardedFieldsFirst("f2", "f3->f0", "f4")}).
 	 * Please refer to the JavaDoc of {@link org.apache.flink.api.common.functions.Function} or Flink's documentation for
 	 * details on field references such as nested fields and wildcard.
 	 * </p>
@@ -202,14 +202,14 @@ public abstract class TwoInputUdfOperator<IN1, IN2, OUT, O extends TwoInputUdfOp
 	 * <p>
 	 * Fields which are unchanged copied from the second input to another position in the output are declared
 	 * by specifying the source field reference in the second input and the target field reference in the output.
-	 * <code>withForwardedFieldsSecond("f0->f2")</code> denotes that the first field of the second input Java tuple is
+	 * {@code withForwardedFieldsSecond("f0->f2")} denotes that the first field of the second input Java tuple is
 	 * unchanged copied to the third field of the Java output tuple. When using a wildcard ("*") ensure that
 	 * the number of declared fields and their types in second input and output type match.
 	 * </p>
 	 *
 	 * <p>
-	 * Multiple forwarded fields can be annotated in one (<code>withForwardedFieldsSecond("f2; f3->f0; f4")</code>)
-	 * or separate Strings (<code>withForwardedFieldsSecond("f2", "f3->f0", "f4")</code>).
+	 * Multiple forwarded fields can be annotated in one ({@code withForwardedFieldsSecond("f2; f3->f0; f4")})
+	 * or separate Strings ({@code withForwardedFieldsSecond("f2", "f3->f0", "f4")}).
 	 * Please refer to the JavaDoc of {@link org.apache.flink.api.common.functions.Function} or Flink's documentation for
 	 * details on field references such as nested fields and wildcard.
 	 * </p>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
index 32ae19c..c74e918 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
@@ -62,7 +62,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Applies an Aggregate transformation on a grouped {@link org.apache.flink.api.java.tuple.Tuple} {@link DataSet}.<br/>
+	 * Applies an Aggregate transformation on a grouped {@link org.apache.flink.api.java.tuple.Tuple} {@link DataSet}.<br>
 	 * <b>Note: Only Tuple DataSets can be aggregated.</b>
 	 * The transformation applies a built-in {@link Aggregations Aggregation} on a specified field 
 	 *   of a Tuple group. Additional aggregation functions can be added to the resulting 
@@ -120,7 +120,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	}
 	
 	/**
-	 * Applies a Reduce transformation on a grouped {@link DataSet}.<br/>
+	 * Applies a Reduce transformation on a grouped {@link DataSet}.<br>
 	 * For each group, the transformation consecutively calls a {@link org.apache.flink.api.common.functions.RichReduceFunction}
 	 *   until only a single element for each group remains. 
 	 * A ReduceFunction combines two elements into one new element of the same type.
@@ -140,7 +140,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	}
 	
 	/**
-	 * Applies a GroupReduce transformation on a grouped {@link DataSet}.<br/>
+	 * Applies a GroupReduce transformation on a grouped {@link DataSet}.<br>
 	 * The transformation calls a {@link org.apache.flink.api.common.functions.RichGroupReduceFunction} for each group of the DataSet.
 	 * A GroupReduceFunction can iterate over all elements of a group and emit any
 	 *   number of output elements including none.
@@ -183,7 +183,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	}
 
 	/**
-	 * Returns a new set containing the first n elements in this grouped {@link DataSet}.<br/>
+	 * Returns a new set containing the first n elements in this grouped {@link DataSet}.<br>
 	 * @param n The desired number of elements for each group.
 	 * @return A GroupReduceOperator that represents the DataSet containing the elements.
 	*/
@@ -196,7 +196,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	}
 
 	/**
-	 * Applies a special case of a reduce transformation (minBy) on a grouped {@link DataSet}.<br/>
+	 * Applies a special case of a reduce transformation (minBy) on a grouped {@link DataSet}.<br>
 	 * The transformation consecutively calls a {@link ReduceFunction} 
 	 * until only a single element remains which is the result of the transformation.
 	 * A ReduceFunction combines two elements into one new element of the same type.
@@ -217,7 +217,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	}
 	
 	/**
-	 * Applies a special case of a reduce transformation (maxBy) on a grouped {@link DataSet}.<br/>
+	 * Applies a special case of a reduce transformation (maxBy) on a grouped {@link DataSet}.<br>
 	 * The transformation consecutively calls a {@link ReduceFunction} 
 	 * until only a single element remains which is the result of the transformation.
 	 * A ReduceFunction combines two elements into one new element of the same type.
@@ -241,8 +241,8 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	// --------------------------------------------------------------------------------------------
 	
 	/**
-	 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}.</br>
-	 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br/>
+	 * Sorts {@link org.apache.flink.api.java.tuple.Tuple} elements within a group on the specified field in the specified {@link Order}.<br>
+	 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br>
 	 * Groups can be sorted by multiple fields by chaining {@link #sortGroup(int, Order)} calls.
 	 * 
 	 * @param field The Tuple field on which the group is sorted.
@@ -263,8 +263,8 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 	}
 	
 	/**
-	 * Sorts Pojos within a group on the specified field in the specified {@link Order}.</br>
-	 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br/>
+	 * Sorts Pojos within a group on the specified field in the specified {@link Order}.<br>
+	 * <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br>
 	 * Groups can be sorted by multiple fields by chaining {@link #sortGroup(String, Order)} calls.
 	 * 
 	 * @param field The Tuple or Pojo field on which the group is sorted.
@@ -285,7 +285,7 @@ public class UnsortedGrouping<T> extends Grouping<T> {
 
 	/**
 	 * Sorts elements within a group on a key extracted by the specified {@link org.apache.flink.api.java.functions.KeySelector}
-	 * in the specified {@link Order}.</br>
+	 * in the specified {@link Order}.<br>
 	 * Chaining {@link #sortGroup(KeySelector, Order)} calls is not supported.
 	 *
 	 * @param keySelector The KeySelector with which the group is sorted.


[02/13] flink git commit: [FLINK-2869] [tests] Port IOManagerPerformanceBenchmark to JMH.

Posted by fh...@apache.org.
[FLINK-2869] [tests] Port IOManagerPerformanceBenchmark to JMH.

This closes #1270.


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

Branch: refs/heads/master
Commit: 3abbcd1eed42c2f7a65243ec6f27d16dbfc235b5
Parents: b654e98
Author: gallenvara <ga...@126.com>
Authored: Tue Oct 20 09:49:34 2015 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 10:23:22 2015 +0100

----------------------------------------------------------------------
 .../IOManagerPerformanceBenchmark.java          | 608 +++++++++++++++++++
 .../IOManagerPerformanceBenchmark.java          | 415 -------------
 2 files changed, 608 insertions(+), 415 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3abbcd1e/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
new file mode 100644
index 0000000..7d31925
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
@@ -0,0 +1,608 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.benchmark.runtime.io.disk.iomanager;
+
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.runtime.io.disk.iomanager.*;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.types.IntValue;
+import org.junit.Assert;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Thread)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class IOManagerPerformanceBenchmark {
+
+	private static final Logger LOG = LoggerFactory.getLogger(IOManagerPerformanceBenchmark.class);
+
+	@Param({"4096", "16384", "524288"})
+	private int segmentSizesAligned;
+
+	@Param({"3862", "16895", "500481"})
+	private int segmentSizesUnaligned;
+
+	@Param({"1", "2", "4", "6"})
+	private int numSegment;
+
+	private static int numBlocks;
+
+	private static final long MEMORY_SIZE = 32 * 1024 * 1024;
+
+	private static final int NUM_INTS_WRITTEN = 100000000;
+
+	private static final AbstractInvokable memoryOwner = new DummyInvokable();
+
+	private MemoryManager memManager;
+
+	private IOManager ioManager;
+
+	private static FileIOChannel.ID fileIOChannel;
+
+	private static File ioManagerTempFile1;
+
+	private static File ioManagerTempFile2;
+
+	private static File speedTestNIOTempFile1;
+
+	private static File speedTestNIOTempFile2;
+
+	private static File speedTestNIOTempFile3;
+
+	private static File speedTestNIOTempFile4;
+
+
+	@Setup
+	public void startup() throws Exception {
+		memManager = new MemoryManager(MEMORY_SIZE, 1);
+		ioManager = new IOManagerAsync();
+		testChannelWriteWithSegments(numSegment);
+		ioManagerTempFile1 = createReadTempFile(segmentSizesAligned);
+		ioManagerTempFile2 = createReadTempFile(segmentSizesUnaligned);
+		speedTestNIOTempFile1 = createSpeedTestNIOTempFile(segmentSizesAligned, true);
+		speedTestNIOTempFile2 = createSpeedTestNIOTempFile(segmentSizesAligned, false);
+		speedTestNIOTempFile3 = createSpeedTestNIOTempFile(segmentSizesUnaligned, true);
+		speedTestNIOTempFile4 = createSpeedTestNIOTempFile(segmentSizesUnaligned, false);
+	}
+
+	@TearDown
+	public void afterTest() throws Exception {
+		ioManagerTempFile1.delete();
+		ioManagerTempFile2.delete();
+		speedTestNIOTempFile1.delete();
+		speedTestNIOTempFile2.delete();
+		speedTestNIOTempFile3.delete();
+		speedTestNIOTempFile4.delete();
+		ioManager.shutdown();
+		Assert.assertTrue("IO Manager has not properly shut down.", ioManager.isProperlyShutDown());
+
+		Assert.assertTrue("Not all memory was returned to the memory manager in the test.", memManager.verifyEmpty());
+		memManager.shutdown();
+		memManager = null;
+	}
+
+// ------------------------------------------------------------------------
+
+	private File createReadTempFile(int bufferSize) throws IOException {
+		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
+		final IntValue rec = new IntValue(0);
+
+		File tempFile = null;
+		DataOutputStream daos = null;
+
+		try {
+			tempFile = new File(tmpChannel.getPath());
+
+			FileOutputStream fos = new FileOutputStream(tempFile);
+			daos = new DataOutputStream(new BufferedOutputStream(fos, bufferSize));
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				rec.setValue(valsLeft);
+				rec.write(new OutputViewDataOutputStreamWrapper(daos));
+			}
+			daos.close();
+			daos = null;
+		}
+		finally {
+			// close if possible
+			if (daos != null) {
+				daos.close();
+			}
+		}
+		return tempFile;
+	}
+
+	@SuppressWarnings("resource")
+	private File createSpeedTestNIOTempFile(int bufferSize, boolean direct) throws IOException
+	{
+		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
+
+		File tempFile = null;
+		FileChannel fs = null;
+
+		try {
+			tempFile = new File(tmpChannel.getPath());
+
+			RandomAccessFile raf = new RandomAccessFile(tempFile, "rw");
+			fs = raf.getChannel();
+
+			ByteBuffer buf = direct ? ByteBuffer.allocateDirect(bufferSize) : ByteBuffer.allocate(bufferSize);
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				if (buf.remaining() < 4) {
+					buf.flip();
+					fs.write(buf);
+					buf.clear();
+				}
+				buf.putInt(valsLeft);
+			}
+
+			if (buf.position() > 0) {
+				buf.flip();
+				fs.write(buf);
+			}
+
+			fs.close();
+			raf.close();
+			fs = null;
+		}
+		finally {
+			// close if possible
+			if (fs != null) {
+				fs.close();
+				fs = null;
+			}
+		}
+		return tempFile;
+	}
+
+	@Benchmark
+	public void speedTestOutputManager() throws Exception
+	{
+		LOG.info("Starting speed test with IO Manager...");
+
+		testChannelWriteWithSegments(numSegment);
+	}
+
+	@Benchmark
+	public void speedTestInputManager() throws Exception
+	{
+		LOG.info("Starting speed test with IO Manager...");
+
+		testChannelReadWithSegments(numSegment);
+	}
+
+	private void testChannelWriteWithSegments(int numSegments) throws Exception
+	{
+		final List<MemorySegment> memory = this.memManager.allocatePages(memoryOwner, numSegments);
+		final FileIOChannel.ID channel = this.ioManager.createChannel();
+
+		BlockChannelWriter<MemorySegment> writer = null;
+
+		try {
+			writer = this.ioManager.createBlockChannelWriter(channel);
+			final ChannelWriterOutputView out = new ChannelWriterOutputView(writer, memory, this.memManager.getPageSize());
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				out.writeInt(valsLeft);
+			}
+
+			fileIOChannel = channel;
+			out.close();
+			numBlocks = out.getBlockCount();
+
+			writer.close();
+			writer = null;
+
+			memManager.release(memory);
+		}
+		finally {
+			if (writer != null) {
+				writer.closeAndDelete();
+			}
+		}
+	}
+
+	private void testChannelReadWithSegments(int numSegments) throws Exception
+	{
+		final List<MemorySegment> memory = this.memManager.allocatePages(memoryOwner, numSegments);
+
+		BlockChannelReader<MemorySegment> reader = null;
+
+		try {
+			reader = ioManager.createBlockChannelReader(fileIOChannel);
+			final ChannelReaderInputView in = new ChannelReaderInputView(reader, memory, numBlocks, false);
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				in.readInt();
+//				Assert.assertTrue(rec.getValue() == valsLeft);
+			}
+
+			in.close();
+			reader.close();
+			reader = null;
+
+			memManager.release(memory);
+		}
+		finally {
+			if (reader != null) {
+				reader.closeAndDelete();
+			}
+		}
+	}
+
+//	@Test
+//	public void speedTestRandomAccessFile() throws IOException {
+//		LOG.info("Starting speed test with java random access file ...");
+//
+//		Channel.ID tmpChannel = ioManager.createChannel();
+//		File tempFile = null;
+//		RandomAccessFile raf = null;
+//
+//		try {
+//			tempFile = new File(tmpChannel.getPath());
+//			raf = new RandomAccessFile(tempFile, "rw");
+//
+//			IntegerRecord rec = new IntegerRecord(0);
+//
+//			long writeStart = System.currentTimeMillis();
+//
+//			int valsLeft = NUM_INTS_WRITTEN;
+//			while (valsLeft-- > 0) {
+//				rec.setValue(valsLeft);
+//				rec.write(raf);
+//			}
+//			raf.close();
+//			raf = null;
+//
+//			long writeElapsed = System.currentTimeMillis() - writeStart;
+//
+//			// ----------------------------------------------------------------
+//
+//			raf = new RandomAccessFile(tempFile, "r");
+//
+//			long readStart = System.currentTimeMillis();
+//
+//			valsLeft = NUM_INTS_WRITTEN;
+//			while (valsLeft-- > 0) {
+//				rec.read(raf);
+//			}
+//			raf.close();
+//			raf = null;
+//
+//			long readElapsed = System.currentTimeMillis() - readStart;
+//
+//
+//			LOG.info("Random Access File: write " + (writeElapsed / 1000) + " secs, read " + (readElapsed / 1000) + " secs.");
+//		}
+//		finally {
+//			// close if possible
+//			if (raf != null) {
+//				raf.close();
+//			}
+//
+//			// try to delete the file
+//			if (tempFile != null) {
+//				tempFile.delete();
+//			}
+//		}
+//	}
+
+	@Benchmark
+	public void speedOutputStreamWithBufferAligned() throws Exception
+	{
+		LOG.info("Starting speed test with java io file stream and ALIGNED buffer sizes ...");
+
+		speedOutputTestStream(segmentSizesAligned);
+	}
+
+	@Benchmark
+	public void speedOutputStreamWithBufferUnaligned() throws Exception
+	{
+		LOG.info("Starting speed test with java io file stream and UNALIGNED buffer sizes ...");
+
+		speedOutputTestStream(segmentSizesUnaligned);
+	}
+
+	@Benchmark
+	public void speedInputStreamWithBufferAligned() throws Exception
+	{
+		LOG.info("Starting speed test with java io file stream and ALIGNED buffer sizes ...");
+
+		speedInputTestStream(segmentSizesAligned);
+	}
+
+	@Benchmark
+	public void speedInputStreamWithBufferUnaligned() throws Exception
+	{
+		LOG.info("Starting speed test with java io file stream and UNALIGNED buffer sizes ...");
+
+		speedInputTestStream(segmentSizesUnaligned);
+	}
+
+	private void speedOutputTestStream(int bufferSize) throws IOException {
+		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
+		final IntValue rec = new IntValue(0);
+
+		File tempFile = null;
+		DataOutputStream daos = null;
+
+		try {
+			tempFile = new File(tmpChannel.getPath());
+
+			FileOutputStream fos = new FileOutputStream(tempFile);
+			daos = new DataOutputStream(new BufferedOutputStream(fos, bufferSize));
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				rec.setValue(valsLeft);
+				rec.write(new OutputViewDataOutputStreamWrapper(daos));
+			}
+			daos.close();
+			daos = null;
+		}
+		finally {
+			// close if possible
+			if (daos != null) {
+				daos.close();
+			}
+			// try to delete the file
+			if (tempFile != null) {
+				tempFile.delete();
+			}
+		}
+	}
+
+	private void speedInputTestStream(int bufferSize) throws IOException {
+		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
+		final IntValue rec = new IntValue(0);
+
+		File tempFile = null;
+		DataInputStream dais = null;
+
+		if (((bufferSize == 4096)||(bufferSize == 16384)||(bufferSize == 524288)))
+		{
+			tempFile = ioManagerTempFile1;
+		}
+		if (((bufferSize == 3862)||(bufferSize == 16895)||(bufferSize == 500481)))
+		{
+			tempFile = ioManagerTempFile2;
+		}
+
+		try {
+			FileInputStream fis = new FileInputStream(tempFile);
+			dais = new DataInputStream(new BufferedInputStream(fis, bufferSize));
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				rec.read(new InputViewDataInputStreamWrapper(dais));
+			}
+			dais.close();
+			dais = null;
+		}
+		finally {
+			// close if possible
+			if (dais != null) {
+				dais.close();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Benchmark
+	public void speedWriteIndirectAndBufferAligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO heap buffers and ALIGNED buffer sizes ...");
+
+		speedWriteTestNIO(segmentSizesAligned, false);
+	}
+
+	@Benchmark
+	public void speedWriteIndirectAndBufferUnaligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO heap buffers and UNALIGNED buffer sizes ...");
+
+		speedWriteTestNIO(segmentSizesUnaligned, false);
+	}
+
+	@Benchmark
+	public void speedWriteDirectAndBufferAligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO direct buffers and ALIGNED buffer sizes ...");
+
+		speedWriteTestNIO(segmentSizesAligned, true);
+	}
+
+	@Benchmark
+	public void speedWriteDirectAndBufferUnaligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO direct buffers and UNALIGNED buffer sizes ...");
+
+		speedWriteTestNIO(segmentSizesUnaligned, true);
+	}
+
+	@Benchmark
+	public void speedReadIndirectAndBufferAligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO heap buffers and ALIGNED buffer sizes ...");
+
+		speedReadTestNIO(segmentSizesAligned, false);
+	}
+
+	@Benchmark
+	public void speedReadIndirectAndBufferUnaligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO heap buffers and UNALIGNED buffer sizes ...");
+
+		speedReadTestNIO(segmentSizesUnaligned, false);
+	}
+
+	@Benchmark
+	public void speedReadDirectAndBufferAligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO direct buffers and ALIGNED buffer sizes ...");
+
+		speedReadTestNIO(segmentSizesAligned, true);
+	}
+
+	@Benchmark
+	public void speedReadDirectAndBufferUnaligned() throws Exception
+	{
+		LOG.info("Starting speed test with java NIO direct buffers and UNALIGNED buffer sizes ...");
+
+		speedReadTestNIO(segmentSizesUnaligned, true);
+	}
+
+
+	@SuppressWarnings("resource")
+	private void speedWriteTestNIO(int bufferSize, boolean direct) throws IOException
+	{
+		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
+
+		File tempFile = null;
+		FileChannel fs = null;
+
+		try {
+			tempFile = new File(tmpChannel.getPath());
+
+			RandomAccessFile raf = new RandomAccessFile(tempFile, "rw");
+			fs = raf.getChannel();
+
+			ByteBuffer buf = direct ? ByteBuffer.allocateDirect(bufferSize) : ByteBuffer.allocate(bufferSize);
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				if (buf.remaining() < 4) {
+					buf.flip();
+					fs.write(buf);
+					buf.clear();
+				}
+				buf.putInt(valsLeft);
+			}
+
+			if (buf.position() > 0) {
+				buf.flip();
+				fs.write(buf);
+			}
+
+			fs.close();
+			raf.close();
+			fs = null;
+		}
+		finally {
+			// close if possible
+			if (fs != null) {
+				fs.close();
+				fs = null;
+			}
+			// try to delete the file
+			if (tempFile != null) {
+				tempFile.delete();
+			}
+		}
+	}
+
+	@SuppressWarnings("resource")
+	private void speedReadTestNIO(int bufferSize, boolean direct) throws IOException
+	{
+		File tempFile = null;
+		FileChannel fs = null;
+
+		if (((bufferSize == 4096)||(bufferSize == 16384)||(bufferSize == 524288))&&(direct))
+		{
+			tempFile = speedTestNIOTempFile1;
+		}
+		if (((bufferSize == 4096)||(bufferSize == 16384)||(bufferSize == 524288))&&(!direct))
+		{
+			tempFile = speedTestNIOTempFile2;
+		}
+		if (((bufferSize == 3862)||(bufferSize == 16895)||(bufferSize == 500481))&&(direct))
+		{
+			tempFile = speedTestNIOTempFile3;
+		}
+		if (((bufferSize == 3862)||(bufferSize == 16895)||(bufferSize == 500481))&&(!direct))
+		{
+			tempFile = speedTestNIOTempFile4;
+		}
+
+		try {
+			ByteBuffer buf = direct ? ByteBuffer.allocateDirect(bufferSize) : ByteBuffer.allocate(bufferSize);
+
+			RandomAccessFile raf = new RandomAccessFile(tempFile, "r");
+			fs = raf.getChannel();
+			buf.clear();
+
+			fs.read(buf);
+			buf.flip();
+
+			int valsLeft = NUM_INTS_WRITTEN;
+			while (valsLeft-- > 0) {
+				if (buf.remaining() < 4) {
+					buf.compact();
+					fs.read(buf);
+					buf.flip();
+				}
+				if (buf.getInt() != valsLeft) {
+					throw new IOException();
+				}
+			}
+
+			fs.close();
+			raf.close();
+
+		}
+		finally {
+			// close if possible
+			if (fs != null) {
+				fs.close();
+				fs = null;
+			}
+		}
+	}
+
+	public static void main(String[] args) throws Exception {
+		Options opt = new OptionsBuilder()
+				.include(IOManagerPerformanceBenchmark.class.getSimpleName())
+				.warmupIterations(2)
+				.measurementIterations(2)
+				.forks(1)
+				.build();
+		new Runner(opt).run();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3abbcd1e/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
deleted file mode 100644
index fd02623..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
+++ /dev/null
@@ -1,415 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.flink.runtime.io.disk.iomanager;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.List;
-
-import org.apache.flink.runtime.operators.testutils.DummyInvokable;
-import org.apache.flink.types.IntValue;
-import org.junit.Assert;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryManager;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-
-public class IOManagerPerformanceBenchmark {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(IOManagerPerformanceBenchmark.class);
-	
-	private static final int[] SEGMENT_SIZES_ALIGNED = { 4096, 16384, 524288 };
-	
-	private static final int[] SEGMENT_SIZES_UNALIGNED = { 3862, 16895, 500481 };
-	
-	private static final int[] NUM_SEGMENTS = { 1, 2, 4, 6 };
-	
-	private static final long MEMORY_SIZE = 32 * 1024 * 1024;
-	
-	private static final int NUM_INTS_WRITTEN = 100000000;
-	
-	
-	private static final AbstractInvokable memoryOwner = new DummyInvokable();
-	
-	private MemoryManager memManager;
-	
-	private IOManager ioManager;
-	
-	
-	@Before
-	public void startup() {
-		memManager = new MemoryManager(MEMORY_SIZE, 1);
-		ioManager = new IOManagerAsync();
-	}
-	
-	@After
-	public void afterTest() throws Exception {
-		ioManager.shutdown();
-		Assert.assertTrue("IO Manager has not properly shut down.", ioManager.isProperlyShutDown());
-		
-		Assert.assertTrue("Not all memory was returned to the memory manager in the test.", memManager.verifyEmpty());
-		memManager.shutdown();
-		memManager = null;
-	}
-	
-// ------------------------------------------------------------------------
-	
-	@Test
-	public void speedTestIOManager() throws Exception
-	{
-		LOG.info("Starting speed test with IO Manager...");
-		
-		for (int num : NUM_SEGMENTS) {
-			testChannelWithSegments(num);
-		}
-	}
-
-	private void testChannelWithSegments(int numSegments) throws Exception
-	{
-		final List<MemorySegment> memory = this.memManager.allocatePages(memoryOwner, numSegments);
-		final FileIOChannel.ID channel = this.ioManager.createChannel();
-		
-		BlockChannelWriter<MemorySegment> writer = null;
-		BlockChannelReader<MemorySegment> reader = null;
-		
-		try {	
-			writer = this.ioManager.createBlockChannelWriter(channel);
-			final ChannelWriterOutputView out = new ChannelWriterOutputView(writer, memory, this.memManager.getPageSize());
-			
-			long writeStart = System.currentTimeMillis();
-			
-			int valsLeft = NUM_INTS_WRITTEN;
-			while (valsLeft-- > 0) {
-				out.writeInt(valsLeft);
-			}
-			
-			out.close();
-			final int numBlocks = out.getBlockCount();
-			writer.close();
-			writer = null;
-			
-			long writeElapsed = System.currentTimeMillis() - writeStart;
-			
-			// ----------------------------------------------------------------
-			
-			reader = ioManager.createBlockChannelReader(channel);
-			final ChannelReaderInputView in = new ChannelReaderInputView(reader, memory, numBlocks, false);
-			
-			long readStart = System.currentTimeMillis();
-			
-			valsLeft = NUM_INTS_WRITTEN;
-			while (valsLeft-- > 0) {
-				in.readInt();
-//				Assert.assertTrue(rec.getValue() == valsLeft);
-			}
-			
-			in.close();
-			reader.close();
-			
-			long readElapsed = System.currentTimeMillis() - readStart;
-			
-			reader.deleteChannel();
-			reader = null;
-			
-			LOG.info("IOManager with " + numSegments + " mem segments: write " + writeElapsed + " msecs, read " + readElapsed + " msecs.");
-			
-			memManager.release(memory);
-		}
-		finally {
-			if (reader != null) {
-				reader.closeAndDelete();
-			}
-			if (writer != null) {
-				writer.closeAndDelete();
-			}
-		}
-	}
-
-//	@Test
-//	public void speedTestRandomAccessFile() throws IOException {
-//		LOG.info("Starting speed test with java random access file ...");
-//		
-//		Channel.ID tmpChannel = ioManager.createChannel();
-//		File tempFile = null;
-//		RandomAccessFile raf = null;
-//		
-//		try {
-//			tempFile = new File(tmpChannel.getPath()); 
-//			raf = new RandomAccessFile(tempFile, "rw");
-//			
-//			IntegerRecord rec = new IntegerRecord(0);
-//			
-//			long writeStart = System.currentTimeMillis();
-//			
-//			int valsLeft = NUM_INTS_WRITTEN;
-//			while (valsLeft-- > 0) {
-//				rec.setValue(valsLeft);
-//				rec.write(raf);
-//			}
-//			raf.close();
-//			raf = null;
-//			
-//			long writeElapsed = System.currentTimeMillis() - writeStart;
-//			
-//			// ----------------------------------------------------------------
-//			
-//			raf = new RandomAccessFile(tempFile, "r");
-//			
-//			long readStart = System.currentTimeMillis();
-//			
-//			valsLeft = NUM_INTS_WRITTEN;
-//			while (valsLeft-- > 0) {
-//				rec.read(raf);
-//			}
-//			raf.close();
-//			raf = null;
-//			
-//			long readElapsed = System.currentTimeMillis() - readStart;
-//			
-//			
-//			LOG.info("Random Access File: write " + (writeElapsed / 1000) + " secs, read " + (readElapsed / 1000) + " secs.");
-//		}
-//		finally {
-//			// close if possible
-//			if (raf != null) {
-//				raf.close();
-//			}
-//			
-//			// try to delete the file
-//			if (tempFile != null) {
-//				tempFile.delete();
-//			}
-//		}
-//	}
-
-	@Test
-	public void speedTestFileStream() throws Exception
-	{
-		LOG.info("Starting speed test with java io file stream and ALIGNED buffer sizes ...");
-		
-		for (int bufferSize : SEGMENT_SIZES_ALIGNED)
-		{
-			speedTestStream(bufferSize);
-		}
-		
-		LOG.info("Starting speed test with java io file stream and UNALIGNED buffer sizes ...");
-		
-		for (int bufferSize : SEGMENT_SIZES_UNALIGNED)
-		{
-			speedTestStream(bufferSize);
-		}
-		
-	}
-		
-	private void speedTestStream(int bufferSize) throws IOException {
-		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
-		final IntValue rec = new IntValue(0);
-		
-		File tempFile = null;
-		DataOutputStream daos = null;
-		DataInputStream dais = null;
-		
-		try {
-			tempFile = new File(tmpChannel.getPath());
-			
-			FileOutputStream fos = new FileOutputStream(tempFile);
-			daos = new DataOutputStream(new BufferedOutputStream(fos, bufferSize));
-			
-			long writeStart = System.currentTimeMillis();
-			
-			int valsLeft = NUM_INTS_WRITTEN;
-			while (valsLeft-- > 0) {
-				rec.setValue(valsLeft);
-				rec.write(new OutputViewDataOutputStreamWrapper(daos));
-			}
-			daos.close();
-			daos = null;
-			
-			long writeElapsed = System.currentTimeMillis() - writeStart;
-			
-			// ----------------------------------------------------------------
-			
-			FileInputStream fis = new FileInputStream(tempFile);
-			dais = new DataInputStream(new BufferedInputStream(fis, bufferSize));
-			
-			long readStart = System.currentTimeMillis();
-			
-			valsLeft = NUM_INTS_WRITTEN;
-			while (valsLeft-- > 0) {
-				rec.read(new InputViewDataInputStreamWrapper(dais));
-			}
-			dais.close();
-			dais = null;
-			
-			long readElapsed = System.currentTimeMillis() - readStart;
-			
-			LOG.info("File-Stream with buffer " + bufferSize + ": write " + writeElapsed + " msecs, read " + readElapsed + " msecs.");
-		}
-		finally {
-			// close if possible
-			if (daos != null) {
-				daos.close();
-			}
-			if (dais != null) {
-				dais.close();
-			}
-			// try to delete the file
-			if (tempFile != null) {
-				tempFile.delete();
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void speedTestNIO() throws Exception
-	{
-		LOG.info("Starting speed test with java NIO heap buffers and ALIGNED buffer sizes ...");
-		
-		for (int bufferSize : SEGMENT_SIZES_ALIGNED)
-		{
-			speedTestNIO(bufferSize, false);
-		}
-		
-		LOG.info("Starting speed test with java NIO heap buffers and UNALIGNED buffer sizes ...");
-		
-		for (int bufferSize : SEGMENT_SIZES_UNALIGNED)
-		{
-			speedTestNIO(bufferSize, false);
-		}
-		
-		LOG.info("Starting speed test with java NIO direct buffers and ALIGNED buffer sizes ...");
-		
-		for (int bufferSize : SEGMENT_SIZES_ALIGNED)
-		{
-			speedTestNIO(bufferSize, true);
-		}
-		
-		LOG.info("Starting speed test with java NIO direct buffers and UNALIGNED buffer sizes ...");
-		
-		for (int bufferSize : SEGMENT_SIZES_UNALIGNED)
-		{
-			speedTestNIO(bufferSize, true);
-		}
-		
-	}
-		
-	@SuppressWarnings("resource")
-	private void speedTestNIO(int bufferSize, boolean direct) throws IOException
-	{
-		final FileIOChannel.ID tmpChannel = ioManager.createChannel();
-		
-		File tempFile = null;
-		FileChannel fs = null;
-		
-		try {
-			tempFile = new File(tmpChannel.getPath());
-			
-			RandomAccessFile raf = new RandomAccessFile(tempFile, "rw");
-			fs = raf.getChannel();
-			
-			ByteBuffer buf = direct ? ByteBuffer.allocateDirect(bufferSize) : ByteBuffer.allocate(bufferSize);
-			
-			long writeStart = System.currentTimeMillis();
-			
-			int valsLeft = NUM_INTS_WRITTEN;
-			while (valsLeft-- > 0) {
-				if (buf.remaining() < 4) {
-					buf.flip();
-					fs.write(buf);
-					buf.clear();
-				}
-				buf.putInt(valsLeft);
-			}
-			
-			if (buf.position() > 0) {
-				buf.flip();
-				fs.write(buf);
-			}
-			
-			fs.close();
-			raf.close();
-			fs = null;
-			
-			long writeElapsed = System.currentTimeMillis() - writeStart;
-			
-			// ----------------------------------------------------------------
-			
-			raf = new RandomAccessFile(tempFile, "r");
-			fs = raf.getChannel();
-			buf.clear();
-			
-			long readStart = System.currentTimeMillis();
-			
-			fs.read(buf);
-			buf.flip();
-			
-			valsLeft = NUM_INTS_WRITTEN;
-			while (valsLeft-- > 0) {
-				if (buf.remaining() < 4) {
-					buf.compact();
-					fs.read(buf);
-					buf.flip();
-				}
-				if (buf.getInt() != valsLeft) {
-					throw new IOException();
-				}
-			}
-			
-			fs.close();
-			raf.close();
-			
-			long readElapsed = System.currentTimeMillis() - readStart;
-			
-			LOG.info("NIO Channel with buffer " + bufferSize + ": write " + writeElapsed + " msecs, read " + readElapsed + " msecs.");
-		}
-		finally {
-			// close if possible
-			if (fs != null) {
-				fs.close();
-				fs = null;
-			}
-			// try to delete the file
-			if (tempFile != null) {
-				tempFile.delete();
-			}
-		}
-	}
-
-}


[09/13] flink git commit: [FLINK-2559] Clean up JavaDocs

Posted by fh...@apache.org.
[FLINK-2559] Clean up JavaDocs

- Remove broken HTML tags like <br/>, <p/>, ...
- close unclosed HTML tags
- replaces special chars by HTML escaping, e.g., '<' by &lt;
- wrap code examples by {@code}
- fix incorrect @see and @link references
- fix incorrect @throws
- fix typos

This closes #1298


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

Branch: refs/heads/master
Commit: 680b5a90de2428b4b6b986f7b40409c2d1d181a7
Parents: ec7bf50
Author: Hubert Czerpak <hu...@gmail.com>
Authored: Thu Oct 22 22:51:25 2015 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 10:52:22 2015 +0100

----------------------------------------------------------------------
 .../org/apache/flink/client/CliFrontend.java    |   3 +-
 .../storm/excamation/ExclamationLocal.java      |  13 +-
 .../storm/excamation/ExclamationTopology.java   |  13 +-
 .../storm/excamation/ExclamationWithBolt.java   |  13 +-
 .../storm/excamation/ExclamationWithSpout.java  |  13 +-
 .../flink/storm/split/SpoutSplitExample.java    |   4 +-
 .../storm/wordcount/BoltTokenizerWordCount.java |   9 +-
 .../wordcount/BoltTokenizerWordCountPojo.java   |   9 +-
 .../BoltTokenizerWordCountWithNames.java        |   9 +-
 .../storm/wordcount/SpoutSourceWordCount.java   |  11 +-
 .../flink/storm/wordcount/WordCountLocal.java   |  11 +-
 .../storm/wordcount/WordCountLocalByName.java   |  11 +-
 .../wordcount/WordCountRemoteByClient.java      |  11 +-
 .../wordcount/WordCountRemoteBySubmitter.java   |  11 +-
 .../storm/wordcount/WordCountTopology.java      |   9 +-
 .../org/apache/flink/storm/api/FlinkClient.java |   2 +-
 .../flink/storm/api/FlinkTopologyBuilder.java   |   4 +-
 .../flink/storm/util/SplitStreamType.java       |   3 +-
 .../flink/storm/wrappers/BoltWrapper.java       |   4 +-
 .../flink/storm/wrappers/SpoutWrapper.java      |  10 +-
 .../model/tweet/entities/Entities.java          |   2 +-
 .../model/tweet/entities/HashTags.java          |   2 +-
 .../model/tweet/entities/Media.java             |   2 +-
 .../model/tweet/entities/Symbol.java            |   2 +-
 .../model/tweet/entities/URL.java               |   2 +-
 .../model/tweet/entities/UserMention.java       |   2 +-
 .../flink/api/common/ExecutionConfig.java       |   2 +-
 .../apache/flink/api/common/ExecutionMode.java  |   8 +-
 .../api/common/accumulators/Histogram.java      |   2 +-
 .../api/common/aggregators/Aggregator.java      |   2 +-
 .../common/distributions/DataDistribution.java  |   2 +-
 .../api/common/functions/CoGroupFunction.java   |   4 +-
 .../api/common/functions/CrossFunction.java     |   4 +-
 .../api/common/functions/FilterFunction.java    |   4 +-
 .../api/common/functions/FlatJoinFunction.java  |   4 +-
 .../api/common/functions/FlatMapFunction.java   |   4 +-
 .../api/common/functions/FoldFunction.java      |   4 +-
 .../common/functions/GroupReduceFunction.java   |   4 +-
 .../api/common/functions/JoinFunction.java      |   4 +-
 .../flink/api/common/functions/MapFunction.java |   4 +-
 .../common/functions/MapPartitionFunction.java  |   4 +-
 .../api/common/functions/ReduceFunction.java    |   4 +-
 .../functions/RichGroupReduceFunction.java      |   2 +-
 .../flink/api/common/io/FileOutputFormat.java   |   2 +-
 .../common/operators/AbstractUdfOperator.java   |   2 +-
 .../api/common/typeutils/TypeComparator.java    |   8 +-
 .../flink/configuration/ConfigConstants.java    |   2 +-
 .../org/apache/flink/core/fs/FileSystem.java    |  18 ++-
 .../apache/flink/core/memory/MemorySegment.java |   4 +-
 .../java/org/apache/flink/types/Record.java     |   2 +-
 .../org/apache/flink/types/StringValue.java     |   4 +-
 .../java/org/apache/flink/util/Visitable.java   |   4 +-
 .../flink/examples/java/clustering/KMeans.java  |   4 +-
 .../clustering/util/KMeansDataGenerator.java    |   2 +
 .../flink/examples/java/distcp/DistCp.java      |   2 +-
 .../examples/java/graph/PageRankBasic.java      |   2 +-
 .../examples/java/ml/LinearRegression.java      |   4 +-
 .../examples/java/relational/TPCHQuery10.java   |   4 +-
 .../examples/java/relational/TPCHQuery3.java    |   4 +-
 .../java/relational/WebLogAnalysis.java         |   8 +-
 .../examples/java/wordcount/PojoExample.java    |   2 +-
 .../examples/java/wordcount/WordCount.java      |   2 +-
 .../examples/java/wordcount/WordCountMeta.java  |   4 +-
 .../java/org/apache/flink/api/java/DataSet.java | 124 +++++++++----------
 .../java/org/apache/flink/api/java/Utils.java   |   1 -
 .../api/java/functions/FunctionAnnotation.java  |  30 ++---
 .../java/hadoop/mapred/HadoopInputFormat.java   |   2 +-
 .../java/hadoop/mapred/HadoopOutputFormat.java  |   2 +-
 .../flink/api/java/io/SplitDataProperties.java  |  16 +--
 .../api/java/operators/CoGroupOperator.java     |  42 +++----
 .../flink/api/java/operators/CrossOperator.java |  30 ++---
 .../flink/api/java/operators/DataSink.java      |   8 +-
 .../flink/api/java/operators/Grouping.java      |   2 +-
 .../flink/api/java/operators/JoinOperator.java  |  52 ++++----
 .../java/operators/SingleInputUdfOperator.java  |   6 +-
 .../api/java/operators/SortedGrouping.java      |  14 +--
 .../api/java/operators/TwoInputUdfOperator.java |  12 +-
 .../api/java/operators/UnsortedGrouping.java    |  22 ++--
 .../operators/join/JoinOperatorSetsBase.java    |  26 ++--
 .../translation/Tuple3WrappingCollector.java    |   2 +-
 .../translation/TupleWrappingCollector.java     |   2 +-
 .../record/functions/FunctionAnnotation.java    |   6 +-
 .../record/io/ExternalProcessInputFormat.java   |   2 +-
 .../flink/api/java/typeutils/AvroTypeInfo.java  |   4 +-
 .../typeutils/runtime/kryo/Serializers.java     |   4 +-
 .../flink/api/java/utils/DataSetUtils.java      |   4 +-
 .../examples/java8/relational/TPCHQuery10.java  |   4 +-
 .../main/java/org/apache/flink/graph/Graph.java |  30 ++---
 .../graph/example/EuclideanGraphWeighing.java   |   3 +-
 .../flink/graph/example/GraphMetrics.java       |   2 +-
 .../flink/graph/example/IncrementalSSSP.java    |   2 +-
 .../flink/graph/example/MusicProfiles.java      |   8 +-
 .../apache/flink/graph/gsa/ApplyFunction.java   |   6 +-
 .../apache/flink/graph/gsa/GatherFunction.java  |   6 +-
 .../org/apache/flink/graph/gsa/Neighbor.java    |   4 +-
 .../org/apache/flink/graph/gsa/SumFunction.java |   6 +-
 .../graph/library/ConnectedComponents.java      |   2 +-
 .../graph/library/GSAConnectedComponents.java   |   2 +-
 .../apache/flink/graph/library/GSAPageRank.java |   2 +-
 .../apache/flink/graph/library/PageRank.java    |   2 +-
 .../flink/graph/spargel/MessagingFunction.java  |   8 +-
 .../graph/spargel/VertexCentricIteration.java   |   2 +-
 .../graph/spargel/VertexUpdateFunction.java     |  16 ++-
 .../flink/optimizer/dag/OptimizerNode.java      |   2 +-
 .../operators/OperatorDescriptorSingle.java     |   2 +-
 .../apache/flink/optimizer/plan/Channel.java    |   1 -
 .../optimizer/postpass/SparseKeySchema.java     |   2 +-
 .../apache/flink/runtime/blob/BlobService.java  |   1 -
 .../runtime/checkpoint/PendingCheckpoint.java   |   1 -
 .../flink/runtime/execution/ExecutionState.java |   4 +-
 .../flink/runtime/filecache/FileCache.java      |   2 +-
 .../instance/SlotSharingGroupAssignment.java    |   4 +-
 .../io/disk/iomanager/BlockChannelReader.java   |   2 +-
 .../io/disk/iomanager/BlockChannelWriter.java   |   2 +-
 .../network/partition/consumer/InputGate.java   |   8 +-
 .../partition/consumer/SingleInputGate.java     |   8 +-
 .../task/IterationIntermediateTask.java         |   2 +-
 .../runtime/jobmanager/scheduler/Scheduler.java |   2 +-
 .../sort/AbstractMergeInnerJoinIterator.java    |   2 +-
 .../runtime/operators/util/BloomFilter.java     |   8 +-
 .../runtime/webmonitor/WebMonitorUtils.java     |   2 +-
 .../runtime/yarn/AbstractFlinkYarnCluster.java  |   4 +-
 .../runtime/zookeeper/StateStorageHelper.java   |   2 +-
 .../zookeeper/ZooKeeperStateHandleStore.java    |   8 +-
 .../mapred/HadoopMapFunction.java               |   2 +-
 .../mapred/HadoopReduceCombineFunction.java     |   2 +-
 .../mapred/HadoopReduceFunction.java            |   4 +-
 .../org/apache/flink/api/java/JarHelper.java    |   2 +-
 .../flink/api/java/table/package-info.java      |   8 +-
 .../connectors/kafka/FlinkKafkaConsumer.java    |   2 +-
 .../kafka/partitioner/FixedPartitioner.java     |  16 +--
 .../examples/iteration/IterateExample.java      |   7 +-
 .../ml/IncrementalLearningSkeleton.java         |   3 +-
 .../socket/SocketTextStreamWordCount.java       |  13 +-
 .../examples/twitter/TwitterStream.java         |  17 ++-
 .../examples/wordcount/PojoExample.java         |   4 +-
 .../streaming/examples/wordcount/WordCount.java |   4 +-
 .../streaming/api/datastream/DataStream.java    |   2 +-
 .../datastream/SingleOutputStreamOperator.java  |   9 +-
 .../environment/StreamExecutionEnvironment.java |  18 +--
 .../functions/source/RichSourceFunction.java    |   1 -
 .../api/functions/source/SourceFunction.java    |  12 +-
 .../api/graph/StreamGraphGenerator.java         |   2 +-
 .../transformations/StreamTransformation.java   |  14 +--
 .../streaming/api/windowing/time/Time.java      |   3 +-
 .../streaming/runtime/tasks/StreamTask.java     |   4 +-
 .../tasks/OneInputStreamTaskTestHarness.java    |   2 +-
 .../runtime/tasks/StreamTaskTestHarness.java    |   2 +-
 .../tasks/TwoInputStreamTaskTestHarness.java    |   2 +-
 .../test/util/MultipleProgramsTestBase.java     |   4 +-
 .../test/recordJobs/wordcount/WordCount.java    |   6 +
 151 files changed, 513 insertions(+), 570 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index c638894..5485030 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -696,7 +696,8 @@ public class CliFrontend {
 	 * Creates a Packaged program from the given command line options.
 	 *
 	 * @return A PackagedProgram (upon success)
-	 * @throws java.io.FileNotFoundException, org.apache.flink.client.program.ProgramInvocationException, java.lang.Throwable
+	 * @throws java.io.FileNotFoundException
+	 * @throws org.apache.flink.client.program.ProgramInvocationException
 	 */
 	protected PackagedProgram buildProgram(ProgramOptions options)
 			throws FileNotFoundException, ProgramInvocationException

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
index 985cd68..56a0125 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
@@ -28,19 +28,16 @@ import org.apache.flink.storm.excamation.operators.ExclamationBolt;
  * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text files in a streaming
  * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology} and submitted to
  * Flink for execution in the same way as to a Storm {@link backtype.storm.LocalCluster}.
- * <p/>
+ * <p>
  * This example shows how to run program directly within Java, thus it cannot be used to submit a
  * {@link backtype.storm.generated.StormTopology} via Flink command line clients (ie, bin/flink).
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>ExclamationLocal &lt;text path&gt; &lt;result path&gt;</code><br/>
+ * <p>
+ * Usage: <code>ExclamationLocal &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from
  * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>run a regular Storm program locally on Flink</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
index 70d25a2..9d94f5c 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
@@ -29,17 +29,14 @@ import org.apache.flink.storm.util.BoltPrintSink;
 
 /**
  * Implements the "Exclamation" program that attaches two exclamation marks to every line of a text files in a streaming
- * fashion. The program is constructed as a regular {@link StormTopology}.
- * <p/>
- * <p/>
+ * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}.
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>Exclamation[Local|RemoteByClient|RemoteBySubmitter] &lt;text path&gt;
- * &lt;result path&gt;</code><br/>
+ * &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>construct a regular Storm topology as Flink program</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
index 01ab907..19fe977 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
@@ -31,17 +31,14 @@ import backtype.storm.utils.Utils;
 
 /**
  * Implements the "Exclamation" program that attaches 3+x exclamation marks to every line of a text files in a streaming
- * fashion. The program is constructed as a regular {@link StormTopology}.
- * <p/>
- * <p/>
+ * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}.
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage:
- * <code>ExclamationWithmBolt &lt;text path&gt; &lt;result path&gt; &lt;number of exclamation marks&gt;</code><br/>
+ * <code>ExclamationWithmBolt &lt;text path&gt; &lt;result path&gt; &lt;number of exclamation marks&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData} with x=2.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>use a Bolt within a Flink Streaming program</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
index 22938e5..a196995 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
@@ -32,16 +32,13 @@ import backtype.storm.utils.Utils;
 
 /**
  * Implements the "Exclamation" program that attaches six exclamation marks to every line of a text files in a streaming
- * fashion. The program is constructed as a regular {@link StormTopology}.
- * <p/>
- * <p/>
+ * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}.
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>ExclamationWithSpout &lt;text path&gt; &lt;result path&gt;</code><br/>
+ * <p>
+ * Usage: <code>ExclamationWithSpout &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>use a Storm spout within a Flink Streaming program</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
index 560fe51..03c87f4 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
@@ -33,14 +33,14 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
 /**
  * Implements a simple example with two declared output streams for the embedded spout.
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>handle multiple output stream of a spout</li>
  * <li>accessing each stream by .split(...) and .select(...)</li>
  * <li>strip wrapper data type SplitStreamType for further processing in Flink</li>
  * </ul>
- * <p/>
+ * <p>
  * This example would work the same way for multiple bolt output streams.
  */
 public class SpoutSplitExample {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
index aa3a075..6ee7fd9 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
@@ -30,15 +30,12 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 /**
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The tokenizer step is performed by a {@link IRichBolt Bolt}.
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>use a Bolt within a Flink Streaming program.</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
index f72acb3..9bdcead 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
@@ -36,15 +36,12 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The tokenizer step is performed by a {@link IRichBolt Bolt}. In contrast to {@link BoltTokenizerWordCount}
  * the tokenizer's input is a POJO type and the single field is accessed by name.
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>how to access attributes by name within a Bolt for POJO type input streams

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
index 7617e95..019f1bc 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
@@ -38,15 +38,12 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The tokenizer step is performed by a {@link IRichBolt Bolt}. In contrast to {@link BoltTokenizerWordCount}
  * the tokenizer's input is a {@link Tuple} type and the single field is accessed by name.
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>how to access attributes by name within a Bolt for {@link Tuple} type input streams

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
index bb451fe..281780e 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
@@ -34,15 +34,12 @@ import org.apache.flink.util.Collector;
 /**
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The used data source is a {@link IRichSpout Spout}.
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>use a Spout within a Flink Streaming program.</li>
@@ -89,7 +86,7 @@ public class SpoutSourceWordCount {
 
 	/**
 	 * Implements the string tokenizer that splits sentences into words as a user-defined FlatMapFunction. The function
-	 * takes a line (String) and splits it into multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
+	 * takes a line (String) and splits it into multiple pairs in the form of "(word,1)" ({@code Tuple2<String, Integer>}).
 	 */
 	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
index 18f49c1..0a7dfa0 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
@@ -29,18 +29,15 @@ import org.apache.flink.storm.api.FlinkTopologyBuilder;
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The program is constructed as a regular {@link StormTopology} and submitted to Flink for execution in the
  * same way as to a Storm {@link LocalCluster}.
- * <p/>
+ * <p>
  * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology}
  * via Flink command line clients (ie, bin/flink).
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCountLocal &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>run a regular Storm program locally on Flink</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
index 71a5e8d..67f4bbe 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
@@ -30,18 +30,15 @@ import org.apache.flink.storm.api.FlinkTopologyBuilder;
  * fashion. The program is constructed as a regular {@link StormTopology} and submitted to Flink for execution in the
  * same way as to a Storm {@link LocalCluster}. In contrast to {@link WordCountLocal} all bolts access the field of
  * input tuples by name instead of index.
- * <p/>
+ * <p>
  * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology}
  * via Flink command line clients (ie, bin/flink).
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCountLocalByName &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>run a regular Storm program locally on Flink

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
index 2e4fb03..8cb7cdd 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
@@ -33,18 +33,15 @@ import org.apache.flink.storm.api.FlinkTopologyBuilder;
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The program is constructed as a regular {@link StormTopology} and submitted to Flink for execution in the
  * same way as to a Storm cluster similar to {@link NimbusClient}. The Flink cluster can be local or remote.
- * <p/>
+ * <p>
  * This example shows how to submit the program via Java, thus it cannot be used to submit a {@link StormTopology} via
  * Flink command line clients (ie, bin/flink).
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCountRemoteByClient &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>submit a regular Storm program to a local or remote Flink cluster.</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
index 173074c..a20843e 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
@@ -30,17 +30,14 @@ import org.apache.flink.storm.api.FlinkTopologyBuilder;
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The program is constructed as a regular {@link StormTopology} and submitted to Flink for execution in the
  * same way as to a Storm cluster similar to {@link StormSubmitter}. The Flink cluster can be local or remote.
- * <p/>
+ * <p>
  * This example shows how to submit the program via Java as well as Flink's command line client (ie, bin/flink).
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage: <code>WordCountRemoteBySubmitter &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>submit a regular Storm program to a local or remote Flink cluster.</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
index 8ee374d..138df65 100644
--- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
@@ -36,16 +36,13 @@ import org.apache.flink.storm.wordcount.operators.WordCountInMemorySpout;
 /**
  * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming
  * fashion. The program is constructed as a regular {@link StormTopology}.
- * <p/>
- * <p/>
+ * <p>
  * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * <p>
  * Usage:
  * <code>WordCount[Local|LocalByName|RemoteByClient|RemoteBySubmitter] &lt;text path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>how to construct a regular Storm topology as Flink program</li>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
index 2be7599..3607fad 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
@@ -138,7 +138,7 @@ public class FlinkClient {
 
 	/**
 	 * Return a reference to itself.
-	 * <p/>
+	 * <p>
 	 * {@link FlinkClient} mimics both, {@link NimbusClient} and {@link Nimbus}{@code .Client}, at once.
 	 *
 	 * @return A reference to itself.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
index 8a88eac..47aa68e 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
@@ -55,8 +55,8 @@ import java.util.Set;
 /**
  * {@link FlinkTopologyBuilder} mimics a {@link TopologyBuilder}, but builds a Flink program instead of a Storm
  * topology. Most methods (except {@link #createTopology()} are copied from the original {@link TopologyBuilder}
- * implementation to ensure equal behavior.<br />
- * <br />
+ * implementation to ensure equal behavior.<br>
+ * <br>
  * <strong>CAUTION: {@link IRichStateSpout StateSpout}s are currently not supported.</strong>
  */
 public class FlinkTopologyBuilder {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
index a4b5f8e..5056795 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
@@ -20,10 +20,11 @@ package org.apache.flink.storm.util;
 import org.apache.flink.streaming.api.datastream.DataStream;
 
 /**
- * Used by {@link org.apache.flink.storm.wrappers.AbstractStormCollector AbstractStormCollector} to wrap
+ * Used by org.apache.flink.storm.wrappers.AbstractStormCollector to wrap
  * output tuples if multiple output streams are declared. For this case, the Flink output data stream must be split via
  * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} using
  * {@link StormStreamSelector}.
+ *
  */
 public class SplitStreamType<T> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
index 12d967a..d9a2f91 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
@@ -42,8 +42,8 @@ import com.google.common.collect.Sets;
  * A {@link BoltWrapper} wraps an {@link IRichBolt} in order to execute the Storm bolt within a Flink Streaming
  * program. It takes the Flink input tuples of type {@code IN} and transforms them into {@link StormTuple}s that the
  * bolt can process. Furthermore, it takes the bolt's output tuples and transforms them into Flink tuples of type
- * {@code OUT} (see {@link AbstractStormCollector} for supported types).<br />
- * <br />
+ * {@code OUT} (see {@link AbstractStormCollector} for supported types).<br>
+ * <br>
  * <strong>CAUTION: currently, only simple bolts are supported! (ie, bolts that do not use the Storm configuration
  * <code>Map</code> or <code>TopologyContext</code> that is provided by the bolt's <code>open(..)</code> method.
  * Furthermore, acking and failing of tuples as well as accessing tuple attributes by field names is not supported so

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/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 e78dd5c..62b36be 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
@@ -38,15 +38,15 @@ import com.google.common.collect.Sets;
 /**
  * A {@link SpoutWrapper} wraps an {@link IRichSpout} in order to execute it within a Flink Streaming program. It
  * takes the spout's output tuples and transforms them into Flink tuples of type {@code OUT} (see
- * {@link SpoutCollector} for supported types).<br />
- * <br />
+ * {@link SpoutCollector} for supported types).<br>
+ * <br>
  * Per default, {@link SpoutWrapper} calls the wrapped spout's {@link IRichSpout#nextTuple() nextTuple()} method in
- * an infinite loop.<br />
+ * an infinite loop.<br>
  * Alternatively, {@link SpoutWrapper} can call {@link IRichSpout#nextTuple() nextTuple()} for a finite number of
  * times and terminate automatically afterwards (for finite input streams). The number of {@code nextTuple()} calls can
  * be specified as a certain number of invocations or can be undefined. In the undefined case, {@link SpoutWrapper}
  * terminates if no record was emitted to the output collector for the first time during a call to
- * {@link IRichSpout#nextTuple() nextTuple()}.<br />
+ * {@link IRichSpout#nextTuple() nextTuple()}.<br>
  * If the given spout implements {@link FiniteSpout} interface and {@link #numberOfInvocations} is not provided or
  * is {@code null}, {@link SpoutWrapper} calls {@link IRichSpout#nextTuple() nextTuple()} method until
  * {@link FiniteSpout#reachedEnd()} returns true.
@@ -258,7 +258,7 @@ public final class SpoutWrapper<OUT> extends RichParallelSourceFunction<OUT> {
 
 	/**
 	 * {@inheritDoc}
-	 * <p/>
+	 * <p>
 	 * Sets the {@link #isRunning} flag to {@code false}.
 	 */
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Entities.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Entities.java b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Entities.java
index d88ea34..8a8f91e 100755
--- a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Entities.java
+++ b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Entities.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 /**
  * Entities which have been parsed out of the text of the
- * {@link package org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet}.
+ * {@link org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet}.
  */
 public class Entities {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/HashTags.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/HashTags.java b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/HashTags.java
index 1900859..ba6f5c0 100755
--- a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/HashTags.java
+++ b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/HashTags.java
@@ -19,7 +19,7 @@ package org.apache.flink.contrib.tweetinputformat.model.tweet.entities;
 
 /**
  * Represents hashtags which have been parsed out of the
- * {@link package org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet} text.
+ * {@link org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet} text.
  */
 
 public class HashTags {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Media.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Media.java b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Media.java
index f006aac..37d2ab8 100755
--- a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Media.java
+++ b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Media.java
@@ -21,7 +21,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 /**
- * Represents media elements uploaded with the {@link package org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet}.
+ * Represents media elements uploaded with the {@link org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet}.
  */
 public class Media {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Symbol.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Symbol.java b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Symbol.java
index db562e8..c2a6dd0 100755
--- a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Symbol.java
+++ b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/Symbol.java
@@ -19,7 +19,7 @@ package org.apache.flink.contrib.tweetinputformat.model.tweet.entities;
 
 /**
  * An array of financial symbols starting with the dollar sign extracted from the
- * {@link package org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet} text.
+ * {@link org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet} text.
  */
 
 public class Symbol {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/URL.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/URL.java b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/URL.java
index 6d0f184..93ddebd 100755
--- a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/URL.java
+++ b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/URL.java
@@ -19,7 +19,7 @@ package org.apache.flink.contrib.tweetinputformat.model.tweet.entities;
 
 /**
  * Represents URLs included in the text of a Tweet or within textual fields of a
- * {@link package org.apache.flink.contrib.tweetinputformat.model.tweet.User.Users} object.
+ * {@link org.apache.flink.contrib.tweetinputformat.model.tweet.entities.UserMention} object.
  */
 public class URL {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/UserMention.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/UserMention.java b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/UserMention.java
index a56f7c7..2a3cf1a 100755
--- a/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/UserMention.java
+++ b/flink-contrib/flink-tweet-inputformat/src/main/java/org/apache/flink/contrib/tweetinputformat/model/tweet/entities/UserMention.java
@@ -19,7 +19,7 @@ package org.apache.flink.contrib.tweetinputformat.model.tweet.entities;
 
 /**
  * Represents other Twitter users mentioned in the text of the
- * {@link package org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet}.
+ * {@link org.apache.flink.contrib.tweetinputformat.model.tweet.Tweet}.
  */
 public class UserMention {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index f38ecb0..b9ebaf7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -677,7 +677,7 @@ public class ExecutionConfig implements Serializable {
 		private static final long serialVersionUID = 1L;
 
 		/**
-		 * Convert UserConfig into a Map<String, String> representation.
+		 * Convert UserConfig into a {@code Map<String, String>} representation.
 		 * This can be used by the runtime, for example for presenting the user config in the web frontend.
 		 *
 		 * @return Key/Value representation of the UserConfig, or null.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
index f3e958e..b1463cc 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionMode.java
@@ -33,10 +33,10 @@ public enum ExecutionMode {
 	 * pipelined manner) are data flows that branch (one data set consumed by multiple
 	 * operations) and re-join later:
 	 * <pre>{@code
-	 *    DataSet data = ...;
-	 *    DataSet mapped1 = data.map(new MyMapper());
-	 *    DataSet mapped2 = data.map(new AnotherMapper());
-	 *    mapped1.join(mapped2).where(...).equalTo(...);
+	 * DataSet data = ...;
+	 * DataSet mapped1 = data.map(new MyMapper());
+	 * DataSet mapped2 = data.map(new AnotherMapper());
+	 * mapped1.join(mapped2).where(...).equalTo(...);
 	 * }</pre>
 	 */
 	PIPELINED,

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java
index f5e959a..f587fee 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java
@@ -23,7 +23,7 @@ import java.util.TreeMap;
 
 /**
  * Histogram accumulator, which builds a histogram in a distributed manner.
- * Implemented as a Integer->Integer TreeMap, so that the entries are sorted
+ * Implemented as a Integer-&gt;Integer TreeMap, so that the entries are sorted
  * according to the values.
  * 
  * This class does not extend to continuous values later, because it makes no

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java b/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java
index 035b591..aabb82f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java
@@ -47,7 +47,7 @@ import org.apache.flink.types.Value;
  *     }
  *     
  *     public boolean filter (Double value) {
- *         if (value > 1000000.0) {
+ *         if (value &gt; 1000000.0) {
  *             agg.aggregate(1);
  *             return false
  *         }

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/distributions/DataDistribution.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/distributions/DataDistribution.java b/flink-core/src/main/java/org/apache/flink/api/common/distributions/DataDistribution.java
index 9bf1abf..ebf1319 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/distributions/DataDistribution.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/distributions/DataDistribution.java
@@ -39,7 +39,7 @@ public interface DataDistribution extends IOReadableWritable, Serializable {
 	 * <p>
 	 * Note: The last bucket's upper bound is actually discarded by many algorithms.
 	 * The last bucket is assumed to hold all values <i>v</i> such that
-	 * {@code v &gt; getBucketBoundary(n-1, n)}, where <i>n</i> is the number of buckets.
+	 * {@code v > getBucketBoundary(n-1, n)}, where <i>n</i> is the number of buckets.
 	 * 
 	 * @param bucketNum The number of the bucket for which to get the upper bound.
 	 * @param totalNumBuckets The number of buckets to split the data into.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java
index 2d7a1d7..1098341 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/CoGroupFunction.java
@@ -28,12 +28,12 @@ import org.apache.flink.util.Collector;
  * If a key is present in only one of the two inputs, it may be that one of the groups is empty.
  * <p>
  * The basic syntax for using CoGoup on two data sets is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> set1 = ...;
  * DataSet<Y> set2 = ...;
  * 
  * set1.coGroup(set2).where(<key-definition>).equalTo(<key-definition>).with(new MyCoGroupFunction());
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * {@code set1} is here considered the first input, {@code set2} the second input.
  * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java
index 0e9d2a5..3660e63 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/CrossFunction.java
@@ -28,12 +28,12 @@ import java.io.Serializable;
  * pair of elements, instead of processing 2-tuples that contain the pairs.
  * <p>
  * The basic syntax for using Cross on two data sets is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> set1 = ...;
  * DataSet<Y> set2 = ...;
  * 
  * set1.cross(set2).with(new MyCrossFunction());
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * {@code set1} is here considered the first input, {@code set2} the second input.
  * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java
index 1f21a96..1a80cb8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FilterFunction.java
@@ -25,11 +25,11 @@ import java.io.Serializable;
  * The predicate decides whether to keep the element, or to discard it.
  * <p>
  * The basic syntax for using a FilterFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  * 
  * DataSet<X> result = input.filter(new MyFilterFunction());
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * <strong>IMPORTANT:</strong> The system assumes that the function does not
  * modify the elements on which the predicate is applied. Violating this assumption

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java
index f99aec4..c79046b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatJoinFunction.java
@@ -34,12 +34,12 @@ import java.io.Serializable;
  * if their key is not contained in the other data set.
  * <p>
  * The basic syntax for using Join on two data sets is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> set1 = ...;
  * DataSet<Y> set2 = ...;
  * 
  * set1.join(set2).where(<key-definition>).equalTo(<key-definition>).with(new MyJoinFunction());
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * {@code set1} is here considered the first input, {@code set2} the second input.
  * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java
index 37c6e83..5b955f7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FlatMapFunction.java
@@ -29,11 +29,11 @@ import java.io.Serializable;
  * use the {@link MapFunction}.
  * <p>
  * The basic syntax for using a FlatMapFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  * 
  * DataSet<Y> result = input.flatMap(new MyFlatMapFunction());
- * </blockquote></pre>
+ * }</pre>
  * 
  * @param <T> Type of the input elements.
  * @param <O> Type of the returned elements.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/FoldFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/FoldFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/FoldFunction.java
index a9c5b2b..c4be00b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/FoldFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/FoldFunction.java
@@ -25,12 +25,12 @@ import java.io.Serializable;
  * a single value, by applying a binary operation to an initial accumulator element every element from a group elements.
  * <p>
  * The basic syntax for using a FoldFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  *
  * X initialValue = ...;
  * DataSet<X> result = input.fold(new MyFoldFunction(), initialValue);
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * Like all functions, the FoldFunction needs to be serializable, as defined in {@link java.io.Serializable}.
  *

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java
index befbee3..36500f6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/GroupReduceFunction.java
@@ -32,11 +32,11 @@ import org.apache.flink.util.Collector;
  * {@link ReduceFunction}.
  * <p>
  * The basic syntax for using a grouped GroupReduceFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  * 
  * DataSet<X> result = input.groupBy(<key-definition>).reduceGroup(new MyGroupReduceFunction());
- * </blockquote></pre>
+ * }</pre>
  * 
  * @param <T> Type of the elements that this function processes.
  * @param <O> The type of the elements returned by the user-defined function.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java
index 085bae9..0032a0f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/JoinFunction.java
@@ -29,12 +29,12 @@ import java.io.Serializable;
  * if their key is not contained in the other data set.
  * <p>
  * The basic syntax for using Join on two data sets is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> set1 = ...;
  * DataSet<Y> set2 = ...;
  * 
  * set1.join(set2).where(<key-definition>).equalTo(<key-definition>).with(new MyJoinFunction());
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * {@code set1} is here considered the first input, {@code set2} the second input.
  * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java
index bbeacc6..45f3d1f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapFunction.java
@@ -28,11 +28,11 @@ import java.io.Serializable;
  * using the {@link FlatMapFunction}.
  * <p>
  * The basic syntax for using a MapFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  * 
  * DataSet<Y> result = input.map(new MyMapFunction());
- * </blockquote></pre>
+ * }</pre>
  * 
  * @param <T> Type of the input elements.
  * @param <O> Type of the returned elements.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/MapPartitionFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/MapPartitionFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapPartitionFunction.java
index 9011d25..d2c0077 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/MapPartitionFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/MapPartitionFunction.java
@@ -31,11 +31,11 @@ import java.io.Serializable;
  * For most of the simple use cases, consider using the {@link MapFunction} or {@link FlatMapFunction}.
  * <p>
  * The basic syntax for a MapPartitionFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  * 
  * DataSet<Y> result = input.mapPartition(new MyMapPartitionFunction());
- * </blockquote></pre>
+ * }</pre>
  * 
  * @param <T> Type of the input elements.
  * @param <O> Type of the returned elements.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java
index 8de7861..1b34047 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/ReduceFunction.java
@@ -32,11 +32,11 @@ import java.io.Serializable;
  * execution strategies.
  * <p>
  * The basic syntax for using a grouped ReduceFunction is as follows:
- * <pre><blockquote>
+ * <pre>{@code
  * DataSet<X> input = ...;
  * 
  * DataSet<X> result = input.groupBy(<key-definition>).reduce(new MyReduceFunction());
- * </blockquote></pre>
+ * }</pre>
  * <p>
  * Like all functions, the ReduceFunction needs to be serializable, as defined in {@link java.io.Serializable}.
  * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
index 48e27d3..b5edc64 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java
@@ -49,7 +49,7 @@ public abstract class RichGroupReduceFunction<IN, OUT> extends AbstractRichFunct
 	 * <p>
 	 * This method is only ever invoked when the subclass of {@link RichGroupReduceFunction}
 	 * adds the {@link Combinable} annotation, or if the <i>combinable</i> flag is set when defining
-	 * the <i>reduceGroup<i> operation via
+	 * the <i>reduceGroup</i> operation via
 	 * org.apache.flink.api.java.operators.GroupReduceOperator#setCombinable(boolean).
 	 * <p>
 	 * Since the reduce function will be called on the result of this method, it is important that this

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
index 6854268..219877d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
@@ -51,7 +51,7 @@ public abstract class FileOutputFormat<IT> extends RichOutputFormat<IT> implemen
 		/** A directory is always created, regardless of number of write tasks. */
 		ALWAYS,	
 		
-		/** A directory is only created for parallel output tasks, i.e., number of output tasks > 1.
+		/** A directory is only created for parallel output tasks, i.e., number of output tasks &gt; 1.
 		 * If number of output tasks = 1, the output is written to a single file. */
 		PARONLY
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java
index 1c79a37..74b0d01 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/AbstractUdfOperator.java
@@ -105,7 +105,7 @@ public abstract class AbstractUdfOperator<OUT, FT extends Function> extends Oper
 	 * Clears all previous broadcast inputs and binds the given inputs as
 	 * broadcast variables of this operator.
 	 * 
-	 * @param inputs The <name, root> pairs to be set as broadcast inputs.
+	 * @param inputs The {@code<name, root>} pairs to be set as broadcast inputs.
 	 */
 	public <T> void setBroadcastVariables(Map<String, Operator<T>> inputs) {
 		this.broadcastInputs.clear();

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
index 3a545e4..cd5da84 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeComparator.java
@@ -76,7 +76,7 @@ public abstract class TypeComparator<T> implements Serializable {
 	 * of the fields from the record, this method may extract those fields.
 	 * <p>
 	 * A typical example for checking the equality of two elements is the following:
-	 * <pre>
+	 * <pre>{@code
 	 * E e1 = ...;
 	 * E e2 = ...;
 	 * 
@@ -84,7 +84,7 @@ public abstract class TypeComparator<T> implements Serializable {
 	 * 
 	 * acc.setReference(e1);
 	 * boolean equal = acc.equalToReference(e2);
-	 * </pre>
+	 * }</pre>
 	 * 
 	 * The rational behind this method is that elements are typically compared using certain features that
 	 * are extracted from them, (such de-serializing as a subset of fields). When setting the
@@ -113,7 +113,7 @@ public abstract class TypeComparator<T> implements Serializable {
 	 * elements {@code e1} and {@code e2} via a comparator, this method can be used the
 	 * following way.
 	 * 
-	 * <pre>
+	 * <pre>{@code
 	 * E e1 = ...;
 	 * E e2 = ...;
 	 * 
@@ -124,7 +124,7 @@ public abstract class TypeComparator<T> implements Serializable {
 	 * acc2.setReference(e2);
 	 * 
 	 * int comp = acc1.compareToReference(acc2);
-	 * </pre>
+	 * }</pre>
 	 * 
 	 * The rational behind this method is that elements are typically compared using certain features that
 	 * are extracted from them, (such de-serializing as a subset of fields). When setting the

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index b64939e..d331548 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -600,7 +600,7 @@ public final class ConfigConstants {
 	public static final boolean DEFAULT_FILESYSTEM_OVERWRITE = false;
 
 	/**
-	 * The default behavior for output directory creating (create only directory when parallelism > 1).
+	 * The default behavior for output directory creating (create only directory when parallelism &gt; 1).
 	 */
 	public static final boolean DEFAULT_FILESYSTEM_ALWAYS_CREATE_DIRECTORY = false;
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
index 185b5f2..7dc92c6 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
@@ -162,8 +162,6 @@ public abstract class FileSystem {
 	 * 
 	 * @return a reference to the {@link FileSystem} instance for accessing the
 	 *         local file system.
-	 * @throws IOException
-	 *         thrown if a reference to the file system instance could not be obtained
 	 */
 	public static FileSystem getLocalFileSystem() {
 		// this should really never fail.
@@ -485,20 +483,20 @@ public abstract class FileSystem {
 	/**
 	 * Initializes output directories on local file systems according to the given write mode.
 	 * 
-	 * WriteMode.CREATE & parallel output:
+	 * WriteMode.CREATE &amp; parallel output:
 	 *  - A directory is created if the output path does not exist.
 	 *  - An existing directory is reused, files contained in the directory are NOT deleted.
 	 *  - An existing file raises an exception.
 	 *    
-	 * WriteMode.CREATE & NONE parallel output:
+	 * WriteMode.CREATE &amp; NONE parallel output:
 	 *  - An existing file or directory raises an exception.
 	 *  
-	 * WriteMode.OVERWRITE & parallel output:
+	 * WriteMode.OVERWRITE &amp; parallel output:
 	 *  - A directory is created if the output path does not exist.
 	 *  - An existing directory is reused, files contained in the directory are NOT deleted.
 	 *  - An existing file is deleted and replaced by a new directory.
 	 *  
-	 * WriteMode.OVERWRITE & NONE parallel output:
+	 * WriteMode.OVERWRITE &amp; NONE parallel output:
 	 *  - An existing file or directory (and all its content) is deleted
 	 * 
 	 * Files contained in an existing directory are not deleted, because multiple instances of a 
@@ -646,19 +644,19 @@ public abstract class FileSystem {
 	/**
 	 * Initializes output directories on distributed file systems according to the given write mode.
 	 * 
-	 * WriteMode.CREATE & parallel output:
+	 * WriteMode.CREATE &amp; parallel output:
 	 *  - A directory is created if the output path does not exist.
 	 *  - An existing file or directory raises an exception.
 	 * 
-	 * WriteMode.CREATE & NONE parallel output:
+	 * WriteMode.CREATE &amp; NONE parallel output:
 	 *  - An existing file or directory raises an exception. 
 	 *    
-	 * WriteMode.OVERWRITE & parallel output:
+	 * WriteMode.OVERWRITE &amp; parallel output:
 	 *  - A directory is created if the output path does not exist.
 	 *  - An existing directory and its content is deleted and a new directory is created.
 	 *  - An existing file is deleted and replaced by a new directory.
 	 *  
-	 *  WriteMode.OVERWRITE & NONE parallel output:
+	 *  WriteMode.OVERWRITE &amp; NONE parallel output:
 	 *  - An existing file or directory is deleted and replaced by a new directory.
 	 * 
 	 * @param outPath Output path that should be prepared.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
index 31d5563..8d13921 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
@@ -210,7 +210,7 @@ public abstract class MemorySegment {
 	}
 
 	/**
-	 * Wraps the chunk of the underlying memory located between <tt>offset<tt> and 
+	 * Wraps the chunk of the underlying memory located between <tt>offset</tt> and
 	 * <tt>length</tt> in a NIO ByteBuffer.
 	 *
 	 * @param offset The offset in the memory segment.
@@ -1220,7 +1220,7 @@ public abstract class MemorySegment {
 	 * @param offset2 Offset of seg2 to start comparing
 	 * @param len Length of the compared memory region
 	 *
-	 * @return 0 if equal, -1 if seg1 < seg2, 1 otherwise
+	 * @return 0 if equal, -1 if seg1 &lt; seg2, 1 otherwise
 	 */
 	public final int compare(MemorySegment seg2, int offset1, int offset2, int len) {
 		while (len >= 8) {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/types/Record.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/types/Record.java b/flink-core/src/main/java/org/apache/flink/types/Record.java
index 24ff979..8ef972f 100644
--- a/flink-core/src/main/java/org/apache/flink/types/Record.java
+++ b/flink-core/src/main/java/org/apache/flink/types/Record.java
@@ -803,7 +803,7 @@ public final class Record implements Value, CopyableValue<Record> {
 	 * Bin-copies fields from a source record to this record. The following caveats apply:
 	 * 
 	 * If the source field is in a modified state, no binary representation will exist yet.
-	 * In that case, this method is equivalent to setField(..., source.getField(..., <class>)). 
+	 * In that case, this method is equivalent to {@code setField(..., source.getField(..., <class>))}.
 	 * In particular, if setValue is called on the source field Value instance, that change 
 	 * will propagate to this record.
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/types/StringValue.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/types/StringValue.java b/flink-core/src/main/java/org/apache/flink/types/StringValue.java
index 2249019..873e1dd 100644
--- a/flink-core/src/main/java/org/apache/flink/types/StringValue.java
+++ b/flink-core/src/main/java/org/apache/flink/types/StringValue.java
@@ -377,7 +377,7 @@ public class StringValue implements NormalizableKey<StringValue>, CharSequence,
 	 * @param prefix The prefix character sequence.
 	 * @param startIndex The position to start checking for the prefix.
 	 * 
-	 * @return True, if this StringValue substring, starting at position <code>startIndex</code> has </code>prefix</code>
+	 * @return True, if this StringValue substring, starting at position <code>startIndex</code> has <code>prefix</code>
 	 *         as its prefix.
 	 */
 	public boolean startsWith(CharSequence prefix, int startIndex) {
@@ -403,7 +403,7 @@ public class StringValue implements NormalizableKey<StringValue>, CharSequence,
 	 * 
 	 * @param prefix The prefix character sequence.
 	 * 
-	 * @return True, if this StringValue has </code>prefix</code> as its prefix.
+	 * @return True, if this StringValue has <code>prefix</code> as its prefix.
 	 */
 	public boolean startsWith(CharSequence prefix) {
 		return startsWith(prefix, 0);

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-core/src/main/java/org/apache/flink/util/Visitable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Visitable.java b/flink-core/src/main/java/org/apache/flink/util/Visitable.java
index 49a7866..559cb96 100644
--- a/flink-core/src/main/java/org/apache/flink/util/Visitable.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Visitable.java
@@ -38,7 +38,7 @@ public interface Visitable<T extends Visitable<T>> {
 	 * and then invokes the post-visit method.
 	 * <p>
 	 * A typical code example is the following:
-	 * <code>
+	 * <pre>{@code
 	 * public void accept(Visitor<Operator> visitor) {
 	 *     boolean descend = visitor.preVisit(this);	
 	 *     if (descend) {
@@ -48,7 +48,7 @@ public interface Visitable<T extends Visitable<T>> {
 	 *         visitor.postVisit(this);
 	 *     }
 	 * }
-	 * </code>
+	 * }</pre>
 	 * 
 	 * @param visitor The visitor to be called with this object as the parameter.
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
index 73f90ca..2db6f65 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
@@ -198,7 +198,7 @@ public class KMeans {
 	//     USER FUNCTIONS
 	// *************************************************************************
 	
-	/** Converts a Tuple2<Double,Double> into a Point. */
+	/** Converts a {@code Tuple2<Double,Double>} into a Point. */
 	@ForwardedFields("0->x; 1->y")
 	public static final class TuplePointConverter implements MapFunction<Tuple2<Double, Double>, Point> {
 
@@ -208,7 +208,7 @@ public class KMeans {
 		}
 	}
 	
-	/** Converts a Tuple3<Integer, Double,Double> into a Centroid. */
+	/** Converts a {@code Tuple3<Integer, Double,Double>} into a Centroid. */
 	@ForwardedFields("0->id; 1->x; 2->y")
 	public static final class TupleCentroidConverter implements MapFunction<Tuple3<Integer, Double, Double>, Centroid> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
index 8f44034..8f48d0a 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
@@ -66,6 +66,8 @@ public class KMeansDataGenerator {
 	 * <li><b>Optional</b> Double: Value range of cluster centers
 	 * <li><b>Optional</b> Long: Random seed
 	 * </ol>
+	 *
+	 * @throws IOException
 	 */
 	public static void main(String[] args) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java
index 08f90a6..8e87892 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java
@@ -51,7 +51,7 @@ import java.util.Map;
  * (see <a href="http://hadoop.apache.org/docs/r1.2.1/distcp.html">http://hadoop.apache.org/docs/r1.2.1/distcp.html</a>)
  * with a dynamic input format
  * Note that this tool does not deal with retriability. Additionally, empty directories are not copied over.
- * <p/>
+ * <p>
  * When running locally, local file systems paths can be used.
  * However, in a distributed environment HDFS paths must be provided both as input and output.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
index f05a15d..7b05158 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/graph/PageRankBasic.java
@@ -53,7 +53,7 @@ import org.apache.flink.examples.java.graph.util.PageRankData;
  * For example <code>"1\n2\n12\n42\n63"</code> gives five pages with IDs 1, 2, 12, 42, and 63.
  * <li>Links are represented as pairs of page IDs which are separated by space 
  * characters. Links are separated by new-line characters.<br>
- * For example <code>"1 2\n2 12\n1 12\n42 63"</code> gives four (directed) links (1)->(2), (2)->(12), (1)->(12), and (42)->(63).<br>
+ * For example <code>"1 2\n2 12\n1 12\n42 63"</code> gives four (directed) links (1)-&gt;(2), (2)-&gt;(12), (1)-&gt;(12), and (42)-&gt;(63).<br>
  * For this simple implementation it is required that each page has at least one incoming and one outgoing link (a page can point to itself).
  * </ul>
  * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
index 341daa6..9c3356c 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/ml/LinearRegression.java
@@ -182,7 +182,7 @@ public class LinearRegression {
 	//     USER FUNCTIONS
 	// *************************************************************************
 
-	/** Converts a Tuple2<Double,Double> into a Data. */
+	/** Converts a {@code Tuple2<Double,Double>} into a Data. */
 	@ForwardedFields("0->x; 1->y")
 	public static final class TupleDataConverter implements MapFunction<Tuple2<Double, Double>, Data> {
 
@@ -192,7 +192,7 @@ public class LinearRegression {
 		}
 	}
 
-	/** Converts a Tuple2<Double,Double> into a Params. */
+	/** Converts a {@code Tuple2<Double,Double>} into a Params. */
 	@ForwardedFields("0->theta0; 1->theta1")
 	public static final class TupleParamsConverter implements MapFunction<Tuple2<Double, Double>,Params> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
index 3dc0472..495ff26 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery10.java
@@ -39,7 +39,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  * This program implements the following SQL equivalent:
  * 
  * <p>
- * <code><pre>
+ * <pre>{@code
  * SELECT 
  *        c_custkey,
  *        c_name, 
@@ -64,7 +64,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
  *        c_acctbal, 
  *        n_name, 
  *        c_address
- * </pre></code>
+ * }</pre>
  *        
  * <p>
  * Compared to the original TPC-H query this version does not print 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
index 9a6e58c..a9e47f6 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/examples/java/relational/TPCHQuery3.java
@@ -42,7 +42,7 @@ import org.apache.flink.api.java.tuple.Tuple4;
  * This program implements the following SQL equivalent:
  *
  * <p>
- * <code><pre>
+ * <pre>{@code
  * SELECT 
  *      l_orderkey, 
  *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
@@ -61,7 +61,7 @@ import org.apache.flink.api.java.tuple.Tuple4;
  *      l_orderkey, 
  *      o_orderdate, 
  *      o_shippriority;
- * </pre></code>
+ * }</pre>
  *
  * <p>
  * Compared to the original TPC-H query this version does not sort the result by revenue


[13/13] flink git commit: [FLINK-2902][web-dashboard] Sort finished jobs by their end time, running jobs by start time

Posted by fh...@apache.org.
[FLINK-2902][web-dashboard] Sort finished jobs by their end time, running jobs by start time

This closes #1296


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

Branch: refs/heads/master
Commit: 676854b92520516a8090eab98b125d6cd9a6b1bf
Parents: 1fd386e
Author: Sachin Goel <sa...@gmail.com>
Authored: Sat Oct 24 16:30:23 2015 +0530
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 11:02:24 2015 +0100

----------------------------------------------------------------------
 .../web-dashboard/app/partials/jobs/completed-jobs.jade          | 2 +-
 .../web-dashboard/app/partials/jobs/running-jobs.jade            | 2 +-
 flink-runtime-web/web-dashboard/app/partials/overview.jade       | 4 ++--
 .../web-dashboard/web/partials/jobs/completed-jobs.html          | 2 +-
 .../web-dashboard/web/partials/jobs/running-jobs.html            | 2 +-
 flink-runtime-web/web-dashboard/web/partials/overview.html       | 4 ++--
 6 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/676854b9/flink-runtime-web/web-dashboard/app/partials/jobs/completed-jobs.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/completed-jobs.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/completed-jobs.jade
index 02c9fd2..c16401d 100644
--- a/flink-runtime-web/web-dashboard/app/partials/jobs/completed-jobs.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/jobs/completed-jobs.jade
@@ -35,7 +35,7 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main
         th Status
 
     tbody
-      tr(ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })")
+      tr(ng-repeat="job in jobs|orderBy:\"'end-time'\":true" ui-sref="single-job.plan.overview({ jobid: job.jid })")
         td {{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
         td {{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
         td {{job.duration}} ms

http://git-wip-us.apache.org/repos/asf/flink/blob/676854b9/flink-runtime-web/web-dashboard/app/partials/jobs/running-jobs.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/running-jobs.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/running-jobs.jade
index 9c02c02..357f159 100644
--- a/flink-runtime-web/web-dashboard/app/partials/jobs/running-jobs.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/jobs/running-jobs.jade
@@ -35,7 +35,7 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main
         th Status
 
     tbody
-      tr(ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })")
+      tr(ng-repeat="job in jobs|orderBy:\"'start-time'\"" ui-sref="single-job.plan.overview({ jobid: job.jid })")
         td {{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
         td {{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
         td {{job.duration}} ms

http://git-wip-us.apache.org/repos/asf/flink/blob/676854b9/flink-runtime-web/web-dashboard/app/partials/overview.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/overview.jade b/flink-runtime-web/web-dashboard/app/partials/overview.jade
index cad0792..0b478d0 100644
--- a/flink-runtime-web/web-dashboard/app/partials/overview.jade
+++ b/flink-runtime-web/web-dashboard/app/partials/overview.jade
@@ -95,7 +95,7 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main
             th Status
 
         tbody
-          tr(ng-repeat="job in runningJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })")
+          tr(ng-repeat="job in runningJobs|orderBy:\"'start-time'\"" ui-sref="single-job.plan.overview({ jobid: job.jid })")
             td {{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
             td {{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
             td {{job.duration}} ms
@@ -123,7 +123,7 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main
             th Status
 
         tbody
-          tr(ng-repeat="job in finishedJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })")
+          tr(ng-repeat="job in finishedJobs|orderBy:\"'end-time'\":true" ui-sref="single-job.plan.overview({ jobid: job.jid })")
             td {{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
             td {{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}
             td {{job.duration}} ms

http://git-wip-us.apache.org/repos/asf/flink/blob/676854b9/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html b/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
index b76278d..bf9f640 100644
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/completed-jobs.html
@@ -35,7 +35,7 @@ limitations under the License.
       </tr>
     </thead>
     <tbody>
-      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+      <tr ng-repeat="job in jobs|orderBy:&quot;'end-time'&quot;:true" ui-sref="single-job.plan.overview({ jobid: job.jid })">
         <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
         <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
         <td>{{job.duration}} ms</td>

http://git-wip-us.apache.org/repos/asf/flink/blob/676854b9/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html b/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
index e175d07..5d95f47 100644
--- a/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
+++ b/flink-runtime-web/web-dashboard/web/partials/jobs/running-jobs.html
@@ -35,7 +35,7 @@ limitations under the License.
       </tr>
     </thead>
     <tbody>
-      <tr ng-repeat="job in jobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+      <tr ng-repeat="job in jobs|orderBy:&quot;'start-time'&quot;" ui-sref="single-job.plan.overview({ jobid: job.jid })">
         <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
         <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
         <td>{{job.duration}} ms</td>

http://git-wip-us.apache.org/repos/asf/flink/blob/676854b9/flink-runtime-web/web-dashboard/web/partials/overview.html
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/partials/overview.html b/flink-runtime-web/web-dashboard/web/partials/overview.html
index ec3c580..295ee27 100644
--- a/flink-runtime-web/web-dashboard/web/partials/overview.html
+++ b/flink-runtime-web/web-dashboard/web/partials/overview.html
@@ -92,7 +92,7 @@ limitations under the License.
           </tr>
         </thead>
         <tbody>
-          <tr ng-repeat="job in runningJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+          <tr ng-repeat="job in runningJobs|orderBy:&quot;'start-time'&quot;" ui-sref="single-job.plan.overview({ jobid: job.jid })">
             <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
             <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
             <td>{{job.duration}} ms</td>
@@ -127,7 +127,7 @@ limitations under the License.
           </tr>
         </thead>
         <tbody>
-          <tr ng-repeat="job in finishedJobs" ui-sref="single-job.plan.overview({ jobid: job.jid })">
+          <tr ng-repeat="job in finishedJobs|orderBy:&quot;'end-time'&quot;:true" ui-sref="single-job.plan.overview({ jobid: job.jid })">
             <td>{{job['start-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
             <td>{{job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss'}}</td>
             <td>{{job.duration}} ms</td>


[11/13] flink git commit: [FLINK-2934] Remove placeholder pages for job.statistics, taskmanager.log and taskmanager.stdout

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/1fd386e7/flink-runtime-web/web-dashboard/web/js/index.js
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/js/index.js b/flink-runtime-web/web-dashboard/web/js/index.js
index d370765..2554a3b 100644
--- a/flink-runtime-web/web-dashboard/web/js/index.js
+++ b/flink-runtime-web/web-dashboard/web/js/index.js
@@ -155,13 +155,6 @@ angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", fu
         controller: 'JobTimelineVertexController'
       }
     }
-  }).state("single-job.statistics", {
-    url: "/statistics",
-    views: {
-      details: {
-        templateUrl: "partials/jobs/job.statistics.html"
-      }
-    }
   }).state("single-job.exceptions", {
     url: "/exceptions",
     views: {
@@ -208,20 +201,6 @@ angular.module('flinkApp', ['ui.router', 'angularMoment']).run(["$rootScope", fu
         templateUrl: "partials/taskmanager/taskmanager.metrics.html"
       }
     }
-  }).state("single-manager.log", {
-    url: "/logfile",
-    views: {
-      details: {
-        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
-      }
-    }
-  }).state("single-manager.stdout", {
-    url: "/stdout",
-    views: {
-      details: {
-        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
-      }
-    }
   }).state("jobmanager", {
     url: "/jobmanager",
     views: {
@@ -1417,4 +1396,4 @@ angular.module('flinkApp').service('TaskManagersService', ["$http", "flinkConfig
   return this;
 }]);
 
-//# sourceMappingURL=data:application/json;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbImluZGV4LmNvZmZlZSIsImluZGV4LmpzIiwiY29tbW9uL2RpcmVjdGl2ZXMuY29mZmVlIiwiY29tbW9uL2RpcmVjdGl2ZXMuanMiLCJjb21tb24vZmlsdGVycy5jb2ZmZWUiLCJjb21tb24vZmlsdGVycy5qcyIsImNvbW1vbi9zZXJ2aWNlcy5jb2ZmZWUiLCJjb21tb24vc2VydmljZXMuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5jdHJsLmNvZmZlZSIsIm1vZHVsZXMvam9ibWFuYWdlci9qb2JtYW5hZ2VyLmN0cmwuanMiLCJtb2R1bGVzL2pvYm1hbmFnZXIvam9ibWFuYWdlci5zdmMuY29mZmVlIiwibW9kdWxlcy9qb2JtYW5hZ2VyL2pvYm1hbmFnZXIuc3ZjLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuY3RybC5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5jdHJsLmpzIiwibW9kdWxlcy9qb2JzL2pvYnMuZGlyLmNvZmZlZSIsIm1vZHVsZXMvam9icy9qb2JzLmRpci5qcyIsIm1vZHVsZXMvam9icy9qb2JzLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL2pvYnMvam9icy5zdmMuanMiLCJtb2R1bGVzL292ZXJ2aWV3L292ZXJ2aWV3LmN0cmwuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5jdHJsLmpzIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuY29mZmVlIiwibW9kdWxlcy9vdmVydmlldy9vdmVydmlldy5zdmMuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5
 hZ2VyLmN0cmwuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5jdHJsLmpzIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuY29mZmVlIiwibW9kdWxlcy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5kaXIuanMiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5jb2ZmZWUiLCJtb2R1bGVzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN2Yy5qcyJdLCJuYW1lcyI6W10sIm1hcHBpbmdzIjoiQUFrQkEsUUFBUSxPQUFPLFlBQVksQ0FBQyxhQUFhLGtCQUl4QyxtQkFBSSxTQUFDLFlBQUQ7RUFDSCxXQUFXLGlCQUFpQjtFQ3JCNUIsT0RzQkEsV0FBVyxjQUFjLFdBQUE7SUFDdkIsV0FBVyxpQkFBaUIsQ0FBQyxXQUFXO0lDckJ4QyxPRHNCQSxXQUFXLGVBQWU7O0lBSTdCLE1BQU0sZUFBZTtFQUNwQixvQkFBb0I7R0FLckIsK0RBQUksU0FBQyxhQUFhLGFBQWEsYUFBYSxXQUF4QztFQUNILFlBQVksYUFBYSxLQUFLLFNBQUMsUUFBRDtJQUM1QixRQUFRLE9BQU8sYUFBYTtJQUU1QixZQUFZO0lDNUJaLE9EOEJBLFVBQVUsV0FBQTtNQzdCUixPRDhCQSxZQUFZO09BQ1osWUFBWTs7RUFFaEIsV0FBVyxXQUFXO0lBQ3BCLFFBQVE7TUFDTixRQUFROzs7RUFZWixXQUFXLGNBQWMsUUFBUTtJQUNoQyxNQUFNO0lBQ04sS0FBSztJQUNMLE1BQU07S0FDSixNQUFNLFNBQVMscUJBQXFCLFFBQVE7RUFFL0MsV0FBVyxRQUFRO0lBQ2xCLFFBQVEsQ0FBQyxXQUFXLFdBQVcsV0FB
 VyxXQUFXLFdBQVcsV0FBVyxXQUMxRSxXQUFXLFdBQVcsV0FBVztJQUNsQyxPQUFPO01BQ04saUJBQWlCO01BQ2pCLE9BQU87UUFDTixZQUFZOzs7SUFHZCxPQUFPO01BQ04sT0FBTztRQUNOLFVBQVU7UUFDVixZQUFZO1FBQ1osZUFBZTs7O0lBR2pCLFNBQVM7TUFDUixhQUFhO01BQ2IsaUJBQWlCO01BQ2pCLFFBQVE7O0lBRVQsUUFBUTtNQUNQLFdBQVc7UUFDVixZQUFZO1FBQ1osVUFBVTs7O0lBR1osT0FBTztNQUNOLGVBQWU7TUFDZixRQUFRO1FBQ1AsT0FBTztVQUNOLFVBQVU7Ozs7SUFJYixPQUFPO01BQ04sbUJBQW1CO01BQ25CLE9BQU87UUFDTixPQUFPO1VBQ04sZUFBZTs7O01BR2pCLFFBQVE7UUFDUCxPQUFPO1VBQ04sVUFBVTs7OztJQUliLGFBQWE7TUFDWixhQUFhO1FBQ1osV0FBVzs7O0lBSWIsYUFBYTs7RUN4Q2QsT0Q0Q0EsV0FBVyxXQUFXLFdBQVc7SUFLbEMsaUNBQU8sU0FBQyx1QkFBRDtFQy9DTixPRGdEQSxzQkFBc0I7SUFJdkIsZ0RBQU8sU0FBQyxnQkFBZ0Isb0JBQWpCO0VBQ04sZUFBZSxNQUFNLFlBQ25CO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGdCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGNBQ0w7SUFBQSxLQUFLO0lBQ
 0wsVUFBVTtJQUNWLE9BQ0U7TUFBQSxNQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLG1CQUNMO0lBQUEsS0FBSztJQUNMLFVBQVU7SUFDVixPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSw0QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsZ0JBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sZ0NBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLGdCQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHVCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sOEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFFBQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0seUJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSx5QkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7UUFDYixZQUFZOzs7S0FFakIsTUFBTSxxQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLGVBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRWpCLE1BQU0sa0JBQ0g7SUFBQSxLQUFLO0
 lBQ0wsT0FDRTtNQUFBLE1BQ0U7UUFBQSxhQUFhO1FBQ2IsWUFBWTs7O0tBRW5CLE1BQU0sMEJBQ0w7SUFBQSxLQUFLO0lBQ0wsT0FDRTtNQUFBLFNBQ0U7UUFBQSxhQUFhOzs7S0FFbEIsTUFBTSxzQkFDTDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsU0FDRTtRQUFBLGFBQWE7OztLQUVsQixNQUFNLHlCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTs7O0tBRWxCLE1BQU0sY0FDSDtJQUFBLEtBQUs7SUFDTCxPQUNFO01BQUEsTUFDRTtRQUFBLGFBQWE7OztLQUVwQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLHFCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7OztLQUVqQixNQUFNLGtCQUNMO0lBQUEsS0FBSztJQUNMLE9BQ0U7TUFBQSxTQUNFO1FBQUEsYUFBYTtRQUNiLFlBQVk7Ozs7RUMxQmxCLE9ENEJBLG1CQUFtQixVQUFVOztBQzFCL0I7QUNoUEEsUUFBUSxPQUFPLFlBSWQsVUFBVSwyQkFBVyxTQUFDLGFBQUQ7RUNyQnBCLE9Ec0JBO0lBQUEsWUFBWTtJQUNaLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsaUJBQWlCLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJNUQsVUFBVSxvQ0F
 Bb0IsU0FBQyxhQUFEO0VDckI3QixPRHNCQTtJQUFBLFNBQVM7SUFDVCxPQUNFO01BQUEsZUFBZTtNQUNmLFFBQVE7O0lBRVYsVUFBVTtJQUVWLE1BQU0sU0FBQyxPQUFPLFNBQVMsT0FBakI7TUNyQkYsT0RzQkYsTUFBTSxnQkFBZ0IsV0FBQTtRQ3JCbEIsT0RzQkYsc0NBQXNDLFlBQVksb0JBQW9CLE1BQU07Ozs7SUFJakYsVUFBVSxpQkFBaUIsV0FBQTtFQ3JCMUIsT0RzQkE7SUFBQSxTQUFTO0lBQ1QsT0FDRTtNQUFBLE9BQU87O0lBRVQsVUFBVTs7O0FDbEJaO0FDcEJBLFFBQVEsT0FBTyxZQUVkLE9BQU8sb0RBQTRCLFNBQUMscUJBQUQ7RUFDbEMsSUFBQTtFQUFBLGlDQUFpQyxTQUFDLE9BQU8sUUFBUSxnQkFBaEI7SUFDL0IsSUFBYyxPQUFPLFVBQVMsZUFBZSxVQUFTLE1BQXREO01BQUEsT0FBTzs7SUNoQlAsT0RrQkEsT0FBTyxTQUFTLE9BQU8sUUFBUSxPQUFPLGdCQUFnQjtNQUFFLE1BQU07OztFQUVoRSwrQkFBK0IsWUFBWSxvQkFBb0I7RUNmL0QsT0RpQkE7SUFFRCxPQUFPLGdCQUFnQixXQUFBO0VDakJ0QixPRGtCQSxTQUFDLE1BQUQ7SUFFRSxJQUFHLE1BQUg7TUNsQkUsT0RrQlcsS0FBSyxRQUFRLFNBQVMsS0FBSyxRQUFRLFdBQVU7V0FBMUQ7TUNoQkUsT0RnQmlFOzs7R0FFdEUsT0FBTyxTQUFTLFdBQUE7RUNkZixPRGVBLFNBQUMsT0FBTyxXQUFSO0lBQ0UsSUFBQSxRQUFBO0lBQUEsSUFBZSxNQUFNLFdBQVcsV0FBVyxDQUFJLFNBQVMsUUFBeEQ7TUFBQSxPQUFPOztJQUNQLElBQWtCLE9BQU8sY0FB
 YSxhQUF0QztNQUFBLFlBQVk7O0lBQ1osUUFBUSxDQUFFLFNBQVMsTUFBTSxNQUFNLE1BQU0sTUFBTTtJQUMzQyxTQUFTLEtBQUssTUFBTSxLQUFLLElBQUksU0FBUyxLQUFLLElBQUk7SUNUL0MsT0RVQSxDQUFDLFFBQVEsS0FBSyxJQUFJLE1BQU0sS0FBSyxNQUFNLFVBQVUsUUFBUSxhQUFhLE1BQU0sTUFBTTs7O0FDUGxGO0FDaEJBLFFBQVEsT0FBTyxZQUVkLFFBQVEsOENBQWUsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDdEIsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFVBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9Ec0JBOztBQ3BCRjtBQ09BLFFBQVEsT0FBTyxZQUVkLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQ25CeEMsT0RvQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7SUFDeEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNsQnRCLE9EbUJBLE9BQU8sV0FBVyxZQUFZOztJQUVqQyxXQUFXLGdFQUE0QixTQUFDLFFBQVEsdUJBQVQ7RUFDdEMsc0JBQXNCLFdBQVcsS0FBSyxTQUFDLE1BQUQ7SUFDcEMsSUFBSSxPQUFBLGNBQUEsTUFBSjtNQUNFLE9BQU8sYUFBYTs7SUNqQnRCLE9Ea0JBLE9BQU8sV0FBVyxTQUFTOztFQ2hCN0IsT0RrQkEsT0FBTyxhQUFhLFdBQUE7SUNqQmxCLE9Ea0JBLHNCQUFzQixXQUFXLEtBQ
 UssU0FBQyxNQUFEO01DakJwQyxPRGtCQSxPQUFPLFdBQVcsU0FBUzs7O0lBRWhDLFdBQVcsb0VBQThCLFNBQUMsUUFBUSx5QkFBVDtFQUN4Qyx3QkFBd0IsYUFBYSxLQUFLLFNBQUMsTUFBRDtJQUN4QyxJQUFJLE9BQUEsY0FBQSxNQUFKO01BQ0UsT0FBTyxhQUFhOztJQ2Z0QixPRGdCQSxPQUFPLFdBQVcsWUFBWTs7RUNkaEMsT0RnQkEsT0FBTyxhQUFhLFdBQUE7SUNmbEIsT0RnQkEsd0JBQXdCLGFBQWEsS0FBSyxTQUFDLE1BQUQ7TUNmeEMsT0RnQkEsT0FBTyxXQUFXLFlBQVk7Ozs7QUNacEM7QUNkQSxRQUFRLE9BQU8sWUFFZCxRQUFRLDBEQUEyQixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNsQyxJQUFBO0VBQUEsU0FBUztFQUVULEtBQUMsYUFBYSxXQUFBO0lBQ1osSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxxQkFDVCxRQUFRLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7TUFDUCxTQUFTO01DcEJULE9EcUJBLFNBQVMsUUFBUTs7SUNuQm5CLE9EcUJBLFNBQVM7O0VDbkJYLE9EcUJBO0lBRUQsUUFBUSx3REFBeUIsU0FBQyxPQUFPLGFBQWEsSUFBckI7RUFDaEMsSUFBQTtFQUFBLE9BQU87RUFFUCxLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksa0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsT0FBTztNQ3RCUCxPRHVCQSxTQUFTLFFBQVE7O0lDckJuQixPRHVCQSxTQUFTOztFQ3JCWCxPRHVCQTtJQUVELFFBQVEsME
 RBQTJCLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQ2xDLElBQUE7RUFBQSxTQUFTO0VBRVQsS0FBQyxhQUFhLFdBQUE7SUFDWixJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLHFCQUNULFFBQVEsU0FBQyxNQUFNLFFBQVEsU0FBUyxRQUF4QjtNQUNQLFNBQVM7TUN4QlQsT0R5QkEsU0FBUyxRQUFROztJQ3ZCbkIsT0R5QkEsU0FBUzs7RUN2QlgsT0R5QkE7O0FDdkJGO0FDdEJBLFFBQVEsT0FBTyxZQUVkLFdBQVcsNkVBQXlCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDbkMsT0FBTyxjQUFjLFdBQUE7SUNuQm5CLE9Eb0JBLE9BQU8sT0FBTyxZQUFZLFFBQVE7O0VBRXBDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ25CckIsT0RvQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUNsQnhDLE9Eb0JBLE9BQU87SUFJUixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VBQ3JDLE9BQU8sY0FBYyxXQUFBO0lDdEJuQixPRHVCQSxPQUFPLE9BQU8sWUFBWSxRQUFROztFQUVwQyxZQUFZLGlCQUFpQixPQUFPO0VBQ3BDLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUN0QnJCLE9EdUJBLFlBQVksbUJBQW1CLE9BQU87O0VDckJ4QyxPRHVCQSxPQUFPO0lBSVIsV0FBVyxxSEFBdUIsU0FBQyxRQUFRLFFBQVEsY0FBYyxhQUFhLFlBQVksYUFBYSxXQUFyRTtFQUNqQyxJQUFBO0VBQUEsUUFBUSxJQUFJO0VBRVosT0FBTyxRQUFRLGFBQWE7RUFDNUIsT0FBTyxNQUFNO0VBQ2I
 sT0FBTyxPQUFPO0VBQ2QsT0FBTyxXQUFXO0VBRWxCLFlBQVksUUFBUSxhQUFhLE9BQU8sS0FBSyxTQUFDLE1BQUQ7SUFDM0MsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPLEtBQUs7SUMxQm5CLE9EMkJBLE9BQU8sV0FBVyxLQUFLOztFQUV6QixZQUFZLFVBQVUsV0FBQTtJQzFCcEIsT0QyQkEsWUFBWSxRQUFRLGFBQWEsT0FBTyxLQUFLLFNBQUMsTUFBRDtNQUMzQyxPQUFPLE1BQU07TUMxQmIsT0Q0QkEsT0FBTyxXQUFXOztLQUVwQixZQUFZO0VDM0JkLE9ENkJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUFDckIsT0FBTyxNQUFNO0lBQ2IsT0FBTyxPQUFPO0lBQ2QsT0FBTyxXQUFXO0lDNUJsQixPRDhCQSxVQUFVLE9BQU87O0lBS3BCLFdBQVcseUVBQXFCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDL0IsUUFBUSxJQUFJO0VBRVosT0FBTyxTQUFTO0VBQ2hCLE9BQU8sZUFBZTtFQUN0QixPQUFPLFlBQVksWUFBWTtFQUUvQixPQUFPLGFBQWEsU0FBQyxRQUFEO0lBQ2xCLElBQUcsV0FBVSxPQUFPLFFBQXBCO01BQ0UsT0FBTyxTQUFTO01BQ2hCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUFDbEIsT0FBTyxlQUFlO01DbEN0QixPRG9DQSxPQUFPLFdBQVc7V0FOcEI7TUFTRSxPQUFPLFNBQVM7TUFDaEIsT0FBTyxlQUFlO01BQ3RCLE9BQU8sU0FBUztNQUNoQixPQUFPLFdBQVc7TUNwQ2xCLE9EcUNBLE9BQU8sZUFBZTs7O0VBRTFCLE9BQU8saUJBQWlCLFdBQUE7SUFDdEIsT0FBTyxTQUFTO0lBQ2hCLE9BQU8s
 ZUFBZTtJQUN0QixPQUFPLFNBQVM7SUFDaEIsT0FBTyxXQUFXO0lDbkNsQixPRG9DQSxPQUFPLGVBQWU7O0VDbEN4QixPRG9DQSxPQUFPLGFBQWEsV0FBQTtJQ25DbEIsT0RvQ0EsT0FBTyxlQUFlLENBQUMsT0FBTzs7SUFJakMsV0FBVyx1REFBNkIsU0FBQyxRQUFRLGFBQVQ7RUFDdkMsUUFBUSxJQUFJO0VBRVosSUFBRyxPQUFPLFdBQVksQ0FBQyxPQUFPLFVBQVUsQ0FBQyxPQUFPLE9BQU8sS0FBdkQ7SUFDRSxZQUFZLFlBQVksT0FBTyxRQUFRLEtBQUssU0FBQyxNQUFEO01DdEMxQyxPRHVDQSxPQUFPLFdBQVc7OztFQ3BDdEIsT0RzQ0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQUNaLElBQUcsT0FBTyxRQUFWO01DckNFLE9Ec0NBLFlBQVksWUFBWSxPQUFPLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUNyQzFDLE9Ec0NBLE9BQU8sV0FBVzs7OztJQUl6QixXQUFXLDJEQUFpQyxTQUFDLFFBQVEsYUFBVDtFQUMzQyxRQUFRLElBQUk7RUFFWixJQUFHLE9BQU8sV0FBWSxDQUFDLE9BQU8sVUFBVSxDQUFDLE9BQU8sT0FBTyxlQUF2RDtJQUNFLFlBQVksZ0JBQWdCLE9BQU8sUUFBUSxLQUFLLFNBQUMsTUFBRDtNQUM5QyxPQUFPLGVBQWUsS0FBSztNQ3RDM0IsT0R1Q0EsT0FBTyxzQkFBc0IsS0FBSzs7O0VDcEN0QyxPRHNDQSxPQUFPLElBQUksVUFBVSxTQUFDLE9BQUQ7SUFDbkIsUUFBUSxJQUFJO0lBQ1osSUFBRyxPQUFPLFFBQVY7TUNyQ0UsT0RzQ0EsWUFBWSxnQkFBZ0IsT0FBTyxRQUFRL
 EtBQUssU0FBQyxNQUFEO1FBQzlDLE9BQU8sZUFBZSxLQUFLO1FDckMzQixPRHNDQSxPQUFPLHNCQUFzQixLQUFLOzs7O0lBSXpDLFdBQVcsbUZBQStCLFNBQUMsUUFBUSxRQUFRLGNBQWMsYUFBL0I7RUFDekMsUUFBUSxJQUFJO0VBRVosWUFBWSxVQUFVLGFBQWEsVUFBVSxLQUFLLFNBQUMsTUFBRDtJQ3RDaEQsT0R1Q0EsT0FBTyxTQUFTOztFQ3JDbEIsT0R1Q0EsT0FBTyxJQUFJLFVBQVUsU0FBQyxPQUFEO0lBQ25CLFFBQVEsSUFBSTtJQ3RDWixPRHVDQSxZQUFZLFVBQVUsYUFBYSxVQUFVLEtBQUssU0FBQyxNQUFEO01DdENoRCxPRHVDQSxPQUFPLFNBQVM7OztJQUlyQixXQUFXLCtFQUEyQixTQUFDLFFBQVEsUUFBUSxjQUFjLGFBQS9CO0VDdkNyQyxPRHdDQSxZQUFZLGlCQUFpQixLQUFLLFNBQUMsTUFBRDtJQ3ZDaEMsT0R3Q0EsT0FBTyxhQUFhOztJQUl2QixXQUFXLHFEQUEyQixTQUFDLFFBQVEsYUFBVDtFQUNyQyxRQUFRLElBQUk7RUN6Q1osT0QyQ0EsT0FBTyxhQUFhLFNBQUMsUUFBRDtJQUNsQixJQUFHLFdBQVUsT0FBTyxRQUFwQjtNQUNFLE9BQU8sU0FBUztNQzFDaEIsT0Q0Q0EsWUFBWSxRQUFRLFFBQVEsS0FBSyxTQUFDLE1BQUQ7UUMzQy9CLE9ENENBLE9BQU8sT0FBTzs7V0FKbEI7TUFPRSxPQUFPLFNBQVM7TUMzQ2hCLE9ENENBLE9BQU8sT0FBTzs7OztBQ3hDcEI7QUNuSEEsUUFBUSxPQUFPLFlBSWQsVUFBVSxxQkFBVSxTQUFDLFFBQUQ7RUNyQm5CLE9Ec0JBO0lBQUEsVUFBVTtJQUVWLE9BQ0U7TU
 FBQSxNQUFNOztJQUVSLE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBO01BQUEsUUFBUSxLQUFLLFdBQVc7TUFFeEIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxPQUFPLEtBQUssU0FBUztNQUVyQyxjQUFjLFNBQUMsTUFBRDtRQUNaLElBQUEsT0FBQSxLQUFBO1FBQUEsR0FBRyxPQUFPLE9BQU8sVUFBVSxLQUFLO1FBRWhDLFdBQVc7UUFFWCxRQUFRLFFBQVEsS0FBSyxVQUFVLFNBQUMsU0FBUyxHQUFWO1VBQzdCLElBQUE7VUFBQSxRQUFRO1lBQ047Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNO2VBRVI7Y0FDRSxPQUFPO2NBQ1AsT0FBTztjQUNQLGFBQWE7Y0FDYixlQUFlLFFBQVEsV0FBVztjQUNsQyxhQUFhLFFBQVEsV0FBVztjQUNoQyxNQUFNOzs7VUFJVixJQUFHLFFBQVEsV0FBVyxjQUFjLEdBQXBDO1lBQ0UsTUFBTSxLQUFLO2NBQ1QsT0FBTztjQUNQLE9BQU87Y0FDUCxhQUFhO2NBQ2IsZUFBZSxRQUFRLFdBQVc7Y0FDbEMsYUFBYSxRQUFRLFdBQVc7Y0FDaEMsTUFBTTs7O1VDdEJSLE9EeUJGLFNBQVMsS0FBSztZQUNaLE9BQU8sTUFBSSxRQUFRLFVBQVEsT0FBSSxRQUFRO1lBQ3ZDLE9BQU87OztRQUdYLFFBQVEsR0FBRyxXQUFXLFFBQ3JCLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBRXZCLFVBQVU7V0FFWCxPQUFPLFVBQ1AsWUFBWSxTQUFDLE9BQUQ7VUM1QlQ
 sT0Q2QkY7V0FFRCxPQUFPO1VBQUUsTUFBTTtVQUFLLE9BQU87VUFBRyxLQUFLO1VBQUcsUUFBUTtXQUM5QyxXQUFXLElBQ1g7UUMxQkMsT0Q0QkYsTUFBTSxHQUFHLE9BQU8sT0FDZixNQUFNLFVBQ04sS0FBSzs7TUFFUixZQUFZLE1BQU07OztJQU1yQixVQUFVLHVCQUFZLFNBQUMsUUFBRDtFQ2hDckIsT0RpQ0E7SUFBQSxVQUFVO0lBRVYsT0FDRTtNQUFBLFVBQVU7TUFDVixPQUFPOztJQUVULE1BQU0sU0FBQyxPQUFPLE1BQU0sT0FBZDtNQUNKLElBQUEsYUFBQSxZQUFBLE9BQUE7TUFBQSxRQUFRLEtBQUssV0FBVztNQUV4QixhQUFhLEtBQUs7TUFDbEIsUUFBUSxRQUFRLE9BQU8sS0FBSyxTQUFTO01BRXJDLGlCQUFpQixTQUFDLE9BQUQ7UUNqQ2IsT0RrQ0YsTUFBTSxRQUFRLFFBQVE7O01BRXhCLGNBQWMsU0FBQyxNQUFEO1FBQ1osSUFBQSxPQUFBLEtBQUE7UUFBQSxHQUFHLE9BQU8sT0FBTyxVQUFVLEtBQUs7UUFFaEMsV0FBVztRQUVYLFFBQVEsUUFBUSxNQUFNLFNBQUMsUUFBRDtVQUNwQixJQUFHLE9BQU8sZ0JBQWdCLENBQUMsR0FBM0I7WUFDRSxJQUFHLE9BQU8sU0FBUSxhQUFsQjtjQ2xDSSxPRG1DRixTQUFTLEtBQ1A7Z0JBQUEsT0FBTztrQkFDTDtvQkFBQSxPQUFPLGVBQWUsT0FBTztvQkFDN0IsT0FBTztvQkFDUCxhQUFhO29CQUNiLGVBQWUsT0FBTztvQkFDdEIsYUFBYSxPQUFPO29CQUNwQixNQUFNLE9BQU87Ozs7bUJBUm5CO2NDckJJLE9EZ0NGLFNBQVMsS0FDUDtnQkFBQSxPQUFPO2tCQUNMO29C
 QUFBLE9BQU8sZUFBZSxPQUFPO29CQUM3QixPQUFPO29CQUNQLGFBQWE7b0JBQ2IsZUFBZSxPQUFPO29CQUN0QixhQUFhLE9BQU87b0JBQ3BCLE1BQU0sT0FBTztvQkFDYixNQUFNLE9BQU87Ozs7Ozs7UUFHdkIsUUFBUSxHQUFHLFdBQVcsUUFBUSxNQUFNLFNBQUMsR0FBRyxHQUFHLE9BQVA7VUFDbEMsSUFBRyxFQUFFLE1BQUw7WUMxQkksT0QyQkYsT0FBTyxHQUFHLDhCQUE4QjtjQUFFLE9BQU8sTUFBTTtjQUFPLFVBQVUsRUFBRTs7O1dBRzdFLFdBQVc7VUFDVixRQUFRLEdBQUcsS0FBSyxPQUFPO1VBR3ZCLFVBQVU7V0FFWCxPQUFPLFFBQ1AsT0FBTztVQUFFLE1BQU07VUFBRyxPQUFPO1VBQUcsS0FBSztVQUFHLFFBQVE7V0FDNUMsV0FBVyxJQUNYLGlCQUNBO1FDMUJDLE9ENEJGLE1BQU0sR0FBRyxPQUFPLE9BQ2YsTUFBTSxVQUNOLEtBQUs7O01BRVIsTUFBTSxPQUFPLE1BQU0sVUFBVSxTQUFDLE1BQUQ7UUFDM0IsSUFBcUIsTUFBckI7VUM3QkksT0Q2QkosWUFBWTs7Ozs7SUFNakIsVUFBVSx3QkFBVyxTQUFDLFVBQUQ7RUM3QnBCLE9EOEJBO0lBQUEsVUFBVTtJQVFWLE9BQ0U7TUFBQSxNQUFNO01BQ04sU0FBUzs7SUFFWCxNQUFNLFNBQUMsT0FBTyxNQUFNLE9BQWQ7TUFDSixJQUFBLFlBQUEsWUFBQSxpQkFBQSxpQkFBQSxZQUFBLFdBQUEsWUFBQSxVQUFBLFdBQUEsNkJBQUEsR0FBQSxhQUFBLHdCQUFBLE9BQUEsaUJBQUEsT0FBQSxnQkFBQSxnQkFBQSxVQUFBLGVBQUEsZUFBQTtNQUFBLElBQUk7TUFDSixXQ
 UFXLEdBQUcsU0FBUztNQUN2QixZQUFZO01BQ1osUUFBUSxNQUFNO01BRWQsaUJBQWlCLEtBQUssV0FBVztNQUNqQyxRQUFRLEtBQUssV0FBVyxXQUFXO01BQ25DLGlCQUFpQixLQUFLLFdBQVc7TUFFakMsWUFBWSxHQUFHLE9BQU87TUFDdEIsYUFBYSxHQUFHLE9BQU87TUFDdkIsV0FBVyxHQUFHLE9BQU87TUFLckIsYUFBYSxLQUFLO01BQ2xCLFFBQVEsUUFBUSxLQUFLLFdBQVcsSUFBSSxNQUFNO01BRTFDLE1BQU0sU0FBUyxXQUFBO1FBQ2IsSUFBQSxXQUFBLElBQUE7UUFBQSxJQUFHLFNBQVMsVUFBVSxNQUF0QjtVQUdFLFlBQVksU0FBUztVQUNyQixLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELEtBQUssVUFBVSxNQUFNLFNBQVMsVUFBVSxPQUFPLFNBQVM7VUFDeEQsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxTQUFTLFVBQVUsQ0FBRSxJQUFJO1VDMUN2QixPRDZDRixXQUFXLEtBQUssYUFBYSxlQUFlLEtBQUssTUFBTSxLQUFLLGFBQWEsU0FBUyxVQUFVOzs7TUFFaEcsTUFBTSxVQUFVLFdBQUE7UUFDZCxJQUFBLFdBQUEsSUFBQTtRQUFBLElBQUcsU0FBUyxVQUFVLE1BQXRCO1VBR0UsU0FBUyxNQUFNLFNBQVMsVUFBVTtVQUNsQyxZQUFZLFNBQVM7VUFDckIsS0FBSyxVQUFVLE1BQU0sU0FBUyxVQUFVLE9BQU8sU0FBUztVQUN4RCxLQUFLLFVBQVUsTUFBTSxTQUFTLFVBQVUsT0FBTyxTQUFTO1VBQ3hELFNBQVMsVUFBVSxDQUFFLElBQUk7VUM1Q3ZCLE9EK0NGLFdBQVcsS0FBSyxhQU
 FhLGVBQWUsS0FBSyxNQUFNLEtBQUssYUFBYSxTQUFTLFVBQVU7OztNQUdoRyxrQkFBa0IsU0FBQyxJQUFEO1FBQ2hCLElBQUE7UUFBQSxhQUFhO1FBQ2IsSUFBRyxDQUFBLEdBQUEsaUJBQUEsVUFBcUIsR0FBQSxrQkFBQSxPQUF4QjtVQUNFLGNBQWM7VUFDZCxJQUFtQyxHQUFBLGlCQUFBLE1BQW5DO1lBQUEsY0FBYyxHQUFHOztVQUNqQixJQUFnRCxHQUFHLGNBQWEsV0FBaEU7WUFBQSxjQUFjLE9BQU8sR0FBRyxZQUFZOztVQUNwQyxJQUFrRCxHQUFHLG1CQUFrQixXQUF2RTtZQUFBLGNBQWMsVUFBVSxHQUFHOztVQUMzQixjQUFjOztRQ3RDZCxPRHVDRjs7TUFJRix5QkFBeUIsU0FBQyxNQUFEO1FDeENyQixPRHlDRCxTQUFRLHFCQUFxQixTQUFRLHlCQUF5QixTQUFRLGFBQWEsU0FBUSxpQkFBaUIsU0FBUSxpQkFBaUIsU0FBUTs7TUFFaEosY0FBYyxTQUFDLElBQUksTUFBTDtRQUNaLElBQUcsU0FBUSxVQUFYO1VDeENJLE9EeUNGO2VBRUcsSUFBRyx1QkFBdUIsT0FBMUI7VUN6Q0QsT0QwQ0Y7ZUFERztVQ3ZDRCxPRDJDQTs7O01BR04sa0JBQWtCLFNBQUMsSUFBSSxNQUFNLE1BQU0sTUFBakI7UUFFaEIsSUFBQSxZQUFBO1FBQUEsYUFBYSx1QkFBdUIsUUFBUSxhQUFhLEdBQUcsS0FBSyx5QkFBeUIsWUFBWSxJQUFJLFFBQVE7UUFHbEgsSUFBRyxTQUFRLFVBQVg7VUFDRSxjQUFjLHFDQUFxQyxHQUFHLFdBQVc7ZUFEbkU7VUFHRSxjQUFjLDJCQUEyQixHQUFHLFdBQVc7O1FBQ3pELElBQUcsR0FBRyxnQkFBZSxJQUF
 yQjtVQUNFLGNBQWM7ZUFEaEI7VUFHRSxXQUFXLEdBQUc7VUFHZCxXQUFXLGNBQWM7VUFDekIsY0FBYywyQkFBMkIsV0FBVzs7UUFHdEQsSUFBRyxHQUFBLGlCQUFBLE1BQUg7VUFDRSxjQUFjLDRCQUE0QixHQUFHLElBQUksTUFBTTtlQUR6RDtVQUtFLElBQStDLHVCQUF1QixPQUF0RTtZQUFBLGNBQWMsU0FBUyxPQUFPOztVQUM5QixJQUFxRSxHQUFHLGdCQUFlLElBQXZGO1lBQUEsY0FBYyxzQkFBc0IsR0FBRyxjQUFjOztVQUNyRCxJQUF3RixHQUFHLGFBQVksV0FBdkc7WUFBQSxjQUFjLG9CQUFvQixjQUFjLEdBQUcscUJBQXFCOzs7UUFHMUUsY0FBYztRQzNDWixPRDRDRjs7TUFHRiw4QkFBOEIsU0FBQyxJQUFJLE1BQU0sTUFBWDtRQUM1QixJQUFBLFlBQUE7UUFBQSxRQUFRLFNBQVM7UUFFakIsYUFBYSxpQkFBaUIsUUFBUSxhQUFhLE9BQU8sYUFBYSxPQUFPO1FDNUM1RSxPRDZDRjs7TUFHRixnQkFBZ0IsU0FBQyxHQUFEO1FBRWQsSUFBQTtRQUFBLElBQUcsRUFBRSxPQUFPLE9BQU0sS0FBbEI7VUFDRSxJQUFJLEVBQUUsUUFBUSxLQUFLO1VBQ25CLElBQUksRUFBRSxRQUFRLEtBQUs7O1FBQ3JCLE1BQU07UUFDTixPQUFNLEVBQUUsU0FBUyxJQUFqQjtVQUNFLE1BQU0sTUFBTSxFQUFFLFVBQVUsR0FBRyxNQUFNO1VBQ2pDLElBQUksRUFBRSxVQUFVLElBQUksRUFBRTs7UUFDeEIsTUFBTSxNQUFNO1FDM0NWLE9ENENGOztNQUVGLGFBQWEsU0FBQyxHQUFHLE1BQU0sSUFBSSxVQUFrQixNQUFNLE1BQXRDO1FDM0NU
 LElBQUksWUFBWSxNQUFNO1VEMkNDLFdBQVc7O1FBRXBDLElBQUcsR0FBRyxPQUFNLEtBQUssa0JBQWpCO1VDekNJLE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLG1CQUFtQixNQUFNO1lBQ3BELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyx1QkFBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksdUJBQXVCLE1BQU07WUFDeEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLFNBQWpCO1VDekNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLFdBQVcsTUFBTTtZQUM1QyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBRXRCLElBQUcsR0FBRyxPQUFNLEtBQUssY0FBakI7VUN6Q0QsT0QwQ0YsRUFBRSxRQUFRLEdBQUcsSUFDWDtZQUFBLE9BQU8sZ0JBQWdCLElBQUksZUFBZSxNQUFNO1lBQ2hELFdBQVc7WUFDWCxTQUFPLFlBQVksSUFBSTs7ZUFFdEIsSUFBRyxHQUFHLE9BQU0sS0FBSyxjQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBTyxnQkFBZ0IsSUFBSSxlQUFlLE1BQU07WUFDaEQsV0FBVztZQUNYLFNBQU8sWUFBWSxJQUFJOztlQUV0QixJQUFHLEdBQUcsT0FBTSxLQUFLLGdCQUFqQjtVQ3pDRCxPRDBDRixFQUFFLFFBQVEsR0FBRyxJQUNYO1lBQUEsT0FBT
 yxnQkFBZ0IsSUFBSSxpQkFBaUIsTUFBTTtZQUNsRCxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7O2VBSnRCO1VDbkNELE9EMENGLEVBQUUsUUFBUSxHQUFHLElBQ1g7WUFBQSxPQUFPLGdCQUFnQixJQUFJLElBQUksTUFBTTtZQUNyQyxXQUFXO1lBQ1gsU0FBTyxZQUFZLElBQUk7Ozs7TUFFN0IsYUFBYSxTQUFDLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTSxjQUFuQztRQUNYLElBQUE7UUFBQSxJQUFPLGNBQWMsUUFBUSxLQUFLLFFBQU8sQ0FBQyxHQUExQztVQ3RDSSxPRHVDRixFQUFFLFFBQVEsS0FBSyxJQUFJLEdBQUcsSUFDcEI7WUFBQSxPQUFPLGdCQUFnQjtZQUN2QixXQUFXO1lBQ1gsV0FBVzs7ZUFKZjtVQU9FLGNBQWMsY0FBYyxNQUFNLEtBQUs7VUFFdkMsSUFBQSxFQUFPLENBQUMsZUFBZSxhQUFhLFFBQVEsWUFBWSxNQUFNLENBQUMsSUFBL0Q7WUFDRSxhQUFhLEtBQUssWUFBWTtZQUM5QixFQUFFLFFBQVEsWUFBWSxJQUNwQjtjQUFBLE9BQU8sZ0JBQWdCLGFBQWE7Y0FDcEMsV0FBVztjQUNYLFNBQU8sWUFBWSxhQUFhOztZQ3RDaEMsT0R3Q0YsRUFBRSxRQUFRLFlBQVksSUFBSSxHQUFHLElBQzNCO2NBQUEsT0FBTyxnQkFBZ0I7Y0FDdkIsV0FBVzs7Ozs7TUFFbkIsa0JBQWtCLFNBQUMsR0FBRyxNQUFKO1FBQ2hCLElBQUEsSUFBQSxlQUFBLFVBQUEsR0FBQSxHQUFBLEtBQUEsTUFBQSxNQUFBLE1BQUEsY0FBQSxNQUFBLEdBQUEsS0FBQSxJQUFBO1FBQUEsZ0JBQWdCO1FBQ2hCLGVBQWU7UUFFZixJQUFHLE
 tBQUEsU0FBQSxNQUFIO1VBRUUsWUFBWSxLQUFLO2VBRm5CO1VBTUUsWUFBWSxLQUFLO1VBQ2pCLFdBQVc7O1FBRWIsS0FBQSxJQUFBLEdBQUEsTUFBQSxVQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE7VUN2Q0ksS0FBSyxVQUFVO1VEd0NqQixPQUFPO1VBQ1AsT0FBTztVQUVQLElBQUcsR0FBRyxlQUFOO1lBQ0UsS0FBUyxJQUFBLFFBQVEsU0FBUyxNQUFNO2NBQUUsWUFBWTtjQUFNLFVBQVU7ZUFBUSxTQUFTO2NBQzdFLFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUztjQUNULFNBQVM7Y0FDVCxTQUFTO2NBQ1QsU0FBUzs7WUFHWCxVQUFVLEdBQUcsTUFBTTtZQUVuQixnQkFBZ0IsSUFBSTtZQUVwQixJQUFRLElBQUEsUUFBUTtZQUNoQixTQUFTLE9BQU8sS0FBSyxLQUFLLEdBQUc7WUFDN0IsT0FBTyxHQUFHLFFBQVE7WUFDbEIsT0FBTyxHQUFHLFFBQVE7WUFFbEIsUUFBUSxRQUFRLGdCQUFnQjs7VUFFbEMsV0FBVyxHQUFHLE1BQU0sSUFBSSxVQUFVLE1BQU07VUFFeEMsY0FBYyxLQUFLLEdBQUc7VUFHdEIsSUFBRyxHQUFBLFVBQUEsTUFBSDtZQUNFLE1BQUEsR0FBQTtZQUFBLEtBQUEsSUFBQSxHQUFBLE9BQUEsSUFBQSxRQUFBLElBQUEsTUFBQSxLQUFBO2NDMUNJLE9BQU8sSUFBSTtjRDJDYixXQUFXLEdBQUcsTUFBTSxJQUFJLGVBQWUsTUFBTTs7OztRQ3RDakQsT0R3Q0Y7O01BR0YsZ0JBQWdCLFNBQUMsTUFBTSxRQUFQO1FBQ2QsSUFBQSxJQUFBLEdBQUE7UUFBQSxLQUFBLEtBQUEsS0FBQSxPQUFBO1VBQ0UsS0FBSyx
 LQUFLLE1BQU07VUFDaEIsSUFBYyxHQUFHLE9BQU0sUUFBdkI7WUFBQSxPQUFPOztVQUdQLElBQUcsR0FBQSxpQkFBQSxNQUFIO1lBQ0UsS0FBQSxLQUFBLEdBQUEsZUFBQTtjQUNFLElBQStCLEdBQUcsY0FBYyxHQUFHLE9BQU0sUUFBekQ7Z0JBQUEsT0FBTyxHQUFHLGNBQWM7Ozs7OztNQUVoQyxZQUFZLFNBQUMsTUFBRDtRQUNWLElBQUEsR0FBQSxVQUFBLFVBQUEsSUFBQSxlQUFBO1FBQUEsSUFBUSxJQUFBLFFBQVEsU0FBUyxNQUFNO1VBQUUsWUFBWTtVQUFNLFVBQVU7V0FBUSxTQUFTO1VBQzVFLFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUztVQUNULFNBQVM7VUFDVCxTQUFTO1VBQ1QsU0FBUzs7UUFHWCxnQkFBZ0IsR0FBRztRQUVuQixXQUFlLElBQUEsUUFBUTtRQUN2QixXQUFXLEtBQUssVUFBVTtRQUUxQixLQUFBLEtBQUEsV0FBQTtVQ2pDSSxLQUFLLFVBQVU7VURrQ2pCLFVBQVUsT0FBTyxhQUFhLElBQUksTUFBTSxLQUFLLFVBQVU7O1FBRXpELFdBQVc7UUFFWCxnQkFBZ0IsS0FBSyxNQUFNLENBQUMsUUFBUSxRQUFRLGdCQUFnQixVQUFVLEVBQUUsUUFBUSxRQUFRLFlBQVk7UUFDcEcsZ0JBQWdCLEtBQUssTUFBTSxDQUFDLFFBQVEsUUFBUSxnQkFBZ0IsV0FBVyxFQUFFLFFBQVEsU0FBUyxZQUFZO1FBRXRHLFNBQVMsTUFBTSxVQUFVLFVBQVUsQ0FBQyxlQUFlO1FBRW5ELFdBQVcsS0FBSyxhQUFhLGVBQWUsZ0JBQWdCLE9BQU8sZ0JBQWdCLGFBQWEsU0FBUyxVQUFVO1FBRW5ILFNBQVMsR0FBRyxRQUFRLFdB
 QUE7VUFDbEIsSUFBQTtVQUFBLEtBQUssR0FBRztVQ25DTixPRG9DRixXQUFXLEtBQUssYUFBYSxlQUFlLEdBQUcsWUFBWSxhQUFhLEdBQUcsUUFBUTs7UUFFckYsU0FBUztRQ25DUCxPRHFDRixXQUFXLFVBQVUsU0FBUyxHQUFHLFNBQVMsU0FBQyxHQUFEO1VDcEN0QyxPRHFDRixNQUFNLFFBQVE7WUFBRSxRQUFROzs7O01BRTVCLE1BQU0sT0FBTyxNQUFNLE1BQU0sU0FBQyxTQUFEO1FBQ3ZCLElBQXNCLFNBQXRCO1VDakNJLE9EaUNKLFVBQVU7Ozs7OztBQzNCaEI7QUMxYUEsUUFBUSxPQUFPLFlBRWQsUUFBUSw4RUFBZSxTQUFDLE9BQU8sYUFBYSxNQUFNLFVBQVUsSUFBSSxVQUF6QztFQUN0QixJQUFBLFlBQUEsYUFBQSxXQUFBLGNBQUEsTUFBQTtFQUFBLGFBQWE7RUFDYixjQUFjO0VBRWQsWUFBWTtFQUNaLE9BQU87SUFDTCxTQUFTO0lBQ1QsVUFBVTtJQUNWLFdBQVc7SUFDWCxRQUFROztFQUdWLGVBQWU7RUFFZixrQkFBa0IsV0FBQTtJQ3JCaEIsT0RzQkEsUUFBUSxRQUFRLGNBQWMsU0FBQyxVQUFEO01DckI1QixPRHNCQTs7O0VBRUosS0FBQyxtQkFBbUIsU0FBQyxVQUFEO0lDcEJsQixPRHFCQSxhQUFhLEtBQUs7O0VBRXBCLEtBQUMscUJBQXFCLFNBQUMsVUFBRDtJQUNwQixJQUFBO0lBQUEsUUFBUSxhQUFhLFFBQVE7SUNuQjdCLE9Eb0JBLGFBQWEsT0FBTyxPQUFPOztFQUU3QixLQUFDLFlBQVksV0FBQTtJQ25CWCxPRG9CQSxDQUVFLGFBQ0EsYUFDQSxXQUNBLFlBQ0EsVUFDQSxhQUNBOztFQUdKLEtBQUMsc
 0JBQXNCLFNBQUMsT0FBRDtJQUNyQixRQUFPLE1BQU07TUFBYixLQUNPO1FDNUJILE9ENEJtQjtNQUR2QixLQUVPO1FDM0JILE9EMkJpQjtNQUZyQixLQUdPO1FDMUJILE9EMEJvQjtNQUh4QixLQUlPO1FDekJILE9EeUJvQjtNQUp4QixLQUtPO1FDeEJILE9Ed0JrQjtNQUx0QixLQU1PO1FDdkJILE9EdUJvQjtNQU54QixLQU9PO1FDdEJILE9Ec0JrQjtNQVB0QixLQVFPO1FDckJILE9EcUJnQjtNQVJwQjtRQ1hJLE9Eb0JHOzs7RUFFVCxLQUFDLGNBQWMsU0FBQyxNQUFEO0lDbEJiLE9EbUJBLFFBQVEsUUFBUSxNQUFNLFNBQUMsTUFBTSxRQUFQO01BQ3BCLElBQUEsRUFBTyxLQUFLLGNBQWMsQ0FBQyxJQUEzQjtRQ2xCRSxPRG1CQSxLQUFLLGNBQWMsS0FBSyxnQkFBZ0IsS0FBSzs7OztFQUVuRCxLQUFDLGtCQUFrQixTQUFDLE1BQUQ7SUFDakIsUUFBUSxRQUFRLEtBQUssVUFBVSxTQUFDLFFBQVEsR0FBVDtNQ2hCN0IsT0RpQkEsT0FBTyxPQUFPOztJQ2ZoQixPRGlCQSxLQUFLLFNBQVMsUUFBUTtNQUNwQixNQUFNO01BQ04sY0FBYyxLQUFLLFdBQVc7TUFDOUIsWUFBWSxLQUFLLFdBQVcsYUFBYTtNQUN6QyxNQUFNOzs7RUFHVixLQUFDLFdBQVcsV0FBQTtJQUNWLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZUFDVCxRQUFRLENBQUEsU0FBQSxPQUFBO01DakJQLE9EaUJPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxRQUFRLFFBQVEsTUFBTSxTQUFDLE1BQU0sU0FBUDtVQUNwQixRQUFPO1
 lBQVAsS0FDTztjQ2hCRCxPRGdCZ0IsS0FBSyxVQUFVLE1BQUMsWUFBWTtZQURsRCxLQUVPO2NDZkQsT0RlaUIsS0FBSyxXQUFXLE1BQUMsWUFBWTtZQUZwRCxLQUdPO2NDZEQsT0Rja0IsS0FBSyxZQUFZLE1BQUMsWUFBWTtZQUh0RCxLQUlPO2NDYkQsT0RhZSxLQUFLLFNBQVMsTUFBQyxZQUFZOzs7UUFFbEQsU0FBUyxRQUFRO1FDWGYsT0RZRjs7T0FUTztJQ0FULE9EV0EsU0FBUzs7RUFFWCxLQUFDLFVBQVUsU0FBQyxNQUFEO0lDVlQsT0RXQSxLQUFLOztFQUVQLEtBQUMsYUFBYSxXQUFBO0lDVlosT0RXQTs7RUFFRixLQUFDLFVBQVUsU0FBQyxPQUFEO0lBQ1QsYUFBYTtJQUNiLFVBQVUsTUFBTSxHQUFHO0lBRW5CLE1BQU0sSUFBSSxVQUFVLE9BQ25CLFFBQVEsQ0FBQSxTQUFBLE9BQUE7TUNaUCxPRFlPLFNBQUMsTUFBTSxRQUFRLFNBQVMsUUFBeEI7UUFDUCxNQUFDLFlBQVksS0FBSztRQUNsQixNQUFDLGdCQUFnQjtRQ1hmLE9EYUYsTUFBTSxJQUFJLFVBQVUsUUFBUSxXQUMzQixRQUFRLFNBQUMsV0FBRDtVQUNQLE9BQU8sUUFBUSxPQUFPLE1BQU07VUFFNUIsYUFBYTtVQ2RYLE9EZ0JGLFVBQVUsSUFBSSxRQUFROzs7T0FWakI7SUNGVCxPRGNBLFVBQVUsSUFBSTs7RUFFaEIsS0FBQyxVQUFVLFNBQUMsUUFBRDtJQUNULElBQUEsVUFBQTtJQUFBLFdBQVcsU0FBQyxRQUFRLE1BQVQ7TUFDVCxJQUFBLEdBQUEsS0FBQSxNQUFBO01BQUEsS0FBQSxJQUFBLEdBQUEsTUFBQSxLQUFBLFFBQUEsSUFBQSxLQUFBLEtBQUE
 7UUNYRSxPQUFPLEtBQUs7UURZWixJQUFlLEtBQUssT0FBTSxRQUExQjtVQUFBLE9BQU87O1FBQ1AsSUFBOEMsS0FBSyxlQUFuRDtVQUFBLE1BQU0sU0FBUyxRQUFRLEtBQUs7O1FBQzVCLElBQWMsS0FBZDtVQUFBLE9BQU87OztNQ0hULE9ES0E7O0lBRUYsV0FBVyxHQUFHO0lBRWQsVUFBVSxJQUFJLFFBQVEsS0FBSyxDQUFBLFNBQUEsT0FBQTtNQ0x6QixPREt5QixTQUFDLE1BQUQ7UUFDekIsSUFBQTtRQUFBLFlBQVksU0FBUyxRQUFRLFdBQVcsS0FBSztRQUU3QyxVQUFVLFNBQVMsTUFBQyxXQUFXO1FDSjdCLE9ETUYsU0FBUyxRQUFROztPQUxRO0lDRTNCLE9ES0EsU0FBUzs7RUFFWCxLQUFDLGFBQWEsU0FBQyxRQUFEO0lBQ1osSUFBQSxHQUFBLEtBQUEsS0FBQTtJQUFBLE1BQUEsV0FBQTtJQUFBLEtBQUEsSUFBQSxHQUFBLE1BQUEsSUFBQSxRQUFBLElBQUEsS0FBQSxLQUFBO01DRkUsU0FBUyxJQUFJO01ER2IsSUFBaUIsT0FBTyxPQUFNLFFBQTlCO1FBQUEsT0FBTzs7O0lBRVQsT0FBTzs7RUFFVCxLQUFDLFlBQVksU0FBQyxVQUFEO0lBQ1gsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FBQ3pCLElBQUE7UUFBQSxTQUFTLE1BQUMsV0FBVztRQ0duQixPRERGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUFlLFdBQVcsaUJBQzlELFFBQVEsU0FBQyxNQUFEO1VBRVAsT0FBTyxXQUFXLEtBQUs7VUNBckIsT0RF
 RixTQUFTLFFBQVE7OztPQVJNO0lDVTNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGNBQWMsU0FBQyxVQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsVUFDbkQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsV0FBVyxLQUFLO1VDQWQsT0RFRixTQUFTLFFBQVE7OztPQVBNO0lDUzNCLE9EQUEsU0FBUzs7RUFFWCxLQUFDLGtCQUFrQixTQUFDLFVBQUQ7SUFDakIsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLFVBQVUsSUFBSSxRQUFRLEtBQUssQ0FBQSxTQUFBLE9BQUE7TUNDekIsT0REeUIsU0FBQyxNQUFEO1FDRXZCLE9EQ0YsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVyxpQkFDOUQsUUFBUSxTQUFDLE1BQUQ7VUFDUCxJQUFBO1VBQUEsZUFBZSxLQUFLO1VDQWxCLE9ERUYsTUFBTSxJQUFJLFVBQVUsV0FBVyxNQUFNLGVBQWUsV0FBVywwQkFDOUQsUUFBUSxTQUFDLE1BQUQ7WUFDUCxJQUFBO1lBQUEsc0JBQXNCLEtBQUs7WUNEekIsT0RHRixTQUFTLFFBQVE7Y0FBRSxNQUFNO2NBQWMsVUFBVTs7Ozs7T0FYNUI7SUNnQjNCLE9ESEEsU0FBUzs7RUFFWCxLQUFDLGlCQUFpQixXQUFBO0lBQ2hCLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxVQUFVLElBQUksUUFBUSxLQUFLLENBQUEsU0FBQSxPQUFBO01DSXpCLE9ES
 nlCLFNBQUMsTUFBRDtRQ0t2QixPREhGLE1BQU0sSUFBSSxVQUFVLFdBQVcsTUFBTSxlQUNwQyxRQUFRLFNBQUMsWUFBRDtVQUNQLFdBQVcsYUFBYTtVQ0d0QixPRERGLFNBQVMsUUFBUTs7O09BTk07SUNXM0IsT0RIQSxTQUFTOztFQ0tYLE9ESEE7O0FDS0Y7QUN4TUEsUUFBUSxPQUFPLFlBRWQsV0FBVywrRkFBc0IsU0FBQyxRQUFRLGlCQUFpQixhQUFhLFdBQVcsYUFBbEQ7RUFDaEMsSUFBQTtFQUFBLE9BQU8sY0FBYyxXQUFBO0lBQ25CLE9BQU8sY0FBYyxZQUFZLFFBQVE7SUNsQnpDLE9EbUJBLE9BQU8sZUFBZSxZQUFZLFFBQVE7O0VBRTVDLFlBQVksaUJBQWlCLE9BQU87RUFDcEMsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2xCckIsT0RtQkEsWUFBWSxtQkFBbUIsT0FBTzs7RUFFeEMsT0FBTztFQUVQLGdCQUFnQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbkJsQyxPRG9CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbkJsQixPRG9CQSxnQkFBZ0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ25CbEMsT0RvQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDbEJkLE9Eb0JBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNuQnJCLE9Eb0JBLFVBQVUsT0FBTzs7O0FDakJyQjtBQ0xBLFFBQVEsT0FBTyxZQUVkLFFBQVEsa0RBQW1CLFNBQUMsT0FBTyxhQUFhLElBQXJCO0VBQzFCLElBQUE7RUFBQSxXQUFXO0VBRVgsS0FBQyxlQUFlLFdBQUE7SUFDZCxJQUFBO0lBQUEsV0FBVyxHQUFHO0lBRWQsTUFBTSxJQUFJLFlBQ1
 QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01BQ1AsV0FBVztNQ3BCWCxPRHFCQSxTQUFTLFFBQVE7O0lDbkJuQixPRHFCQSxTQUFTOztFQ25CWCxPRHFCQTs7QUNuQkY7QUNJQSxRQUFRLE9BQU8sWUFFZCxXQUFXLDJGQUE2QixTQUFDLFFBQVEscUJBQXFCLFdBQVcsYUFBekM7RUFDdkMsSUFBQTtFQUFBLG9CQUFvQixlQUFlLEtBQUssU0FBQyxNQUFEO0lDbEJ0QyxPRG1CQSxPQUFPLFdBQVc7O0VBRXBCLFVBQVUsVUFBVSxXQUFBO0lDbEJsQixPRG1CQSxvQkFBb0IsZUFBZSxLQUFLLFNBQUMsTUFBRDtNQ2xCdEMsT0RtQkEsT0FBTyxXQUFXOztLQUNwQixZQUFZO0VDakJkLE9EbUJBLE9BQU8sSUFBSSxZQUFZLFdBQUE7SUNsQnJCLE9EbUJBLFVBQVUsT0FBTzs7SUFFcEIsV0FBVyxrSEFBK0IsU0FBQyxRQUFRLGNBQWMsMEJBQTBCLFdBQVcsYUFBNUQ7RUFDekMsSUFBQTtFQUFBLE9BQU8sVUFBVTtFQUNqQix5QkFBeUIsWUFBWSxhQUFhLGVBQWUsS0FBSyxTQUFDLE1BQUQ7SUNqQnBFLE9Ea0JFLE9BQU8sVUFBVSxLQUFLOztFQUV4QixVQUFVLFVBQVUsV0FBQTtJQ2pCcEIsT0RrQkUseUJBQXlCLFlBQVksYUFBYSxlQUFlLEtBQUssU0FBQyxNQUFEO01DakJ0RSxPRGtCRSxPQUFPLFVBQVUsS0FBSzs7S0FDeEIsWUFBWTtFQ2hCaEIsT0RrQkUsT0FBTyxJQUFJLFlBQVksV0FBQTtJQ2pCdkIsT0RrQkUsVUFBVSxPQUFPOzs7QUNmdkI7QUNWQSxRQUFRLE9BQU8sWUFFZCxVQUFVLGFBQWEsV0FBQTtFQ25CdEI
 sT0RvQkE7SUFDRSxNQUFNLFNBQUMsT0FBTyxTQUFTLE9BQWpCO01BQ0osSUFBQSxpQkFBQSxjQUFBLFNBQUEsU0FBQSxTQUFBLFNBQUEsZUFBQTtNQUFBLGVBQWUsV0FBQTtRQUNiLElBQUcsTUFBTSxRQUFPLFdBQWhCO1VDbEJFLE9EbUJBO2VBREY7VUNoQkUsT0RtQkE7OztNQUVKLGdCQUFnQixXQUFBO1FBQ2QsSUFBRyxNQUFNLFFBQU8sV0FBaEI7VUNqQkUsT0RrQkE7ZUFERjtVQ2ZFLE9Ea0JBOzs7TUFFSixVQUFVLFdBQUE7UUNoQlIsT0RpQkEsa0JBQWtCLE1BQU07O01BQzFCLFVBQVUsV0FBQTtRQ2ZSLE9EZ0JBLGlCQUFpQixNQUFNOztNQUN6QixVQUFVLFdBQUE7UUNkUixPRGVBLHFCQUFxQixNQUFNOztNQUM3QixVQUFVLFdBQUE7UUNiUixPRGNBOztNQUVGLGtCQUFrQixXQUFBO1FDYmhCLE9EYXNCO1VBQ3RCLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFBQyxNQUFNO1lBQWdCLFVBQVU7O1VBQ3hDLE9BQU87WUFBQyxNQUFNOztVQUNkLE9BQU87WUFDTCxPQUFPO2NBQUMsTUFBTTs7WUFDZCxLQUFVLE1BQU0sUUFBTyxZQUFsQixJQUFBLEtBQUE7WUFDTCxLQUFZLE1BQU0sUUFBTyxZQUFwQixNQUFBLEtBQUE7O1VBRVAsUUFBUTtZQUNOO2NBQUMsTUFBTTtjQUFpQixJQUFJO2NBQVcsTUFBTTtjQUFJLE9BQU87ZUFDeEQ7Y0FBQyxNQUFNO2NBQWdCLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztlQUN2RDtjQUFDLE1BQU07Y0FBb0IsSUFBSTtjQUFXLE1BQU07Y0FBSSxPQUFPO2VBQzNEO2NBQUMsTUFB
 TTtjQUFhLElBQUk7Y0FBVyxNQUFNO2NBQUksT0FBTztjQUFXLGNBQWM7OztVQUUvRSxRQUFRO1lBQUMsU0FBUzs7VUFDbEIsU0FBUztZQUFDLFFBQVE7O1VBQ2xCLFdBQVc7WUFBQyxTQUFTOztVQUNyQixTQUFTO1lBQUMsU0FBUzs7OztNQUdyQixJQUFJLFFBQUEsZ0JBQUEsTUFBSjtRQUNFLFFBQVEsV0FBVzs7TUFFckIsTUFBTSxPQUFPLE1BQU0sTUFBTSxTQUFDLE9BQUQ7UUN1QnZCLE9EdEJBLGFBQWE7O01Dd0JmLE9EckJBLGVBQWUsU0FBQyxPQUFEO1FDc0JiLE9EckJFLENBQUEsU0FBQyxPQUFEO1VBQ0EsSUFBQSxPQUFBLFNBQUE7VUFBQSxZQUFZLE1BQU07VUFDbEIsUUFBUSxRQUFRO1VBQ2hCLElBQUcsTUFBTSxRQUFPLFdBQWhCO1lDdUJFLE9EdEJBLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLEtBQUssUUFBUSxNQUNsRSxNQUFNO2lCQUhYO1lBS0UsVUFBVTtZQUNWLE1BQU0sSUFBSSxXQUFXLFNBQVMsQ0FDNUIsV0FBVyxFQUFFLENBQUMsTUFBTSxRQUFRLE9BQU8sV0FBVyxRQUFRLFNBQVMsUUFBUSxNQUN0RSxNQUFNO1lBQ1QsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQUFRLE1BQ3RFLE1BQU07WUNpQlQsT0RoQkEsTUFBTSxJQUFJLFdBQVcsU0FBUyxDQUM1QixXQUFXLEVBQUUsQ0FBQyxNQUFNLFFBQVEsT0FBTyxXQUFXLFFBQVEsU0FBUyxRQ
 UFRLE1BQ3RFLE1BQU07O1dBakJWOzs7OztBQ3NDWDtBQzdGQSxRQUFRLE9BQU8sWUFFZCxRQUFRLHNEQUF1QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUM5QixLQUFDLGVBQWUsV0FBQTtJQUNkLElBQUE7SUFBQSxXQUFXLEdBQUc7SUFFZCxNQUFNLElBQUksZ0JBQ1QsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DcEJQLE9EcUJBLFNBQVMsUUFBUSxLQUFLOztJQ25CeEIsT0RxQkEsU0FBUzs7RUNuQlgsT0RxQkE7SUFFRCxRQUFRLDJEQUE0QixTQUFDLE9BQU8sYUFBYSxJQUFyQjtFQUNuQyxLQUFDLGNBQWMsU0FBQyxlQUFEO0lBQ2IsSUFBQTtJQUFBLFdBQVcsR0FBRztJQUVkLE1BQU0sSUFBSSxrQkFBa0IsZUFDM0IsUUFBUSxTQUFDLE1BQU0sUUFBUSxTQUFTLFFBQXhCO01DdEJQLE9EdUJBLFNBQVMsUUFBUSxLQUFLOztJQ3JCeEIsT0R1QkEsU0FBUzs7RUNyQlgsT0R1QkE7O0FDckJGIiwiZmlsZSI6ImluZGV4LmpzIiwic291cmNlc0NvbnRlbnQiOlsiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbG
 VcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyXG4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcsIFsndWkucm91dGVyJywgJ2FuZ3VsYXJNb21lbnQnXSlcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5ydW4gKCRyb290U2NvcGUpIC0+XHJcbiAgJHJvb3RTY29wZS5zaWR
 lYmFyVmlzaWJsZSA9IGZhbHNlXHJcbiAgJHJvb3RTY29wZS5zaG93U2lkZWJhciA9IC0+XHJcbiAgICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gISRyb290U2NvcGUuc2lkZWJhclZpc2libGVcclxuICAgICRyb290U2NvcGUuc2lkZWJhckNsYXNzID0gJ2ZvcmNlLXNob3cnXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4udmFsdWUgJ2ZsaW5rQ29uZmlnJywge1xyXG4gIFwicmVmcmVzaC1pbnRlcnZhbFwiOiAxMDAwMFxyXG59XHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4ucnVuIChKb2JzU2VydmljZSwgTWFpblNlcnZpY2UsIGZsaW5rQ29uZmlnLCAkaW50ZXJ2YWwpIC0+XHJcbiAgTWFpblNlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGNvbmZpZykgLT5cclxuICAgIGFuZ3VsYXIuZXh0ZW5kIGZsaW5rQ29uZmlnLCBjb25maWdcclxuXHJcbiAgICBKb2JzU2VydmljZS5saXN0Sm9icygpXHJcblxyXG4gICAgJGludGVydmFsIC0+XHJcbiAgICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKClcclxuICAgICwgZmxpbmtDb25maWdbXCJyZWZyZXNoLWludGVydmFsXCJdXHJcblxyXG4gIEhpZ2hjaGFydHMuc2V0T3B0aW9ucyh7XHJcbiAgICBnbG9iYWw6IHtcclxuICAgICAgdXNlVVRDOiBmYWxzZVxyXG4gICAgfVxyXG4gIH0pXHJcblxyXG4gICNcclxuICAjIEdyaWQtbGlnaHQgdGhlbWUgZm9yIEhpZ2hj
 aGFydHMgSlNcclxuICAjIEBhdXRob3IgVG9yc3RlaW4gSG9uc2lcclxuICAjXHJcbiAgIyBUYWtlbiBmcm9tIGh0dHBzOi8vZ2l0aHViLmNvbS9oaWdoc2xpZGUtc29mdHdhcmUvaGlnaGNoYXJ0cy5jb21cclxuICAjXHJcblxyXG5cclxuICBIaWdoY2hhcnRzLmNyZWF0ZUVsZW1lbnQoJ2xpbmsnLCB7XHJcbiAgXHRocmVmOiAnLy9mb250cy5nb29nbGVhcGlzLmNvbS9jc3M/ZmFtaWx5PURvc2lzOjQwMCw2MDAnLFxyXG4gIFx0cmVsOiAnc3R5bGVzaGVldCcsXHJcbiAgXHR0eXBlOiAndGV4dC9jc3MnXHJcbiAgfSwgbnVsbCwgZG9jdW1lbnQuZ2V0RWxlbWVudHNCeVRhZ05hbWUoJ2hlYWQnKVswXSk7XHJcblxyXG4gIEhpZ2hjaGFydHMudGhlbWUgPSB7XHJcbiAgXHRjb2xvcnM6IFtcIiM3Y2I1ZWNcIiwgXCIjZjdhMzVjXCIsIFwiIzkwZWU3ZVwiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCIsIFwiI2ZmMDA2NlwiLCBcIiNlZWFhZWVcIixcclxuICBcdFx0XCIjNTVCRjNCXCIsIFwiI0RGNTM1M1wiLCBcIiM3Nzk4QkZcIiwgXCIjYWFlZWVlXCJdLFxyXG4gIFx0Y2hhcnQ6IHtcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiBudWxsLFxyXG4gIFx0XHRzdHlsZToge1xyXG4gIFx0XHRcdGZvbnRGYW1pbHk6IFwiRG9zaXMsIHNhbnMtc2VyaWZcIlxyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dGl0bGU6IHtcclxuICBcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRmb250U2l6ZTogJzE2cHgnLFxyXG4gIFx0XHRcd
 GZvbnRXZWlnaHQ6ICdib2xkJyxcclxuICBcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0dG9vbHRpcDoge1xyXG4gIFx0XHRib3JkZXJXaWR0aDogMCxcclxuICBcdFx0YmFja2dyb3VuZENvbG9yOiAncmdiYSgyMTksMjE5LDIxNiwwLjgpJyxcclxuICBcdFx0c2hhZG93OiBmYWxzZVxyXG4gIFx0fSxcclxuICBcdGxlZ2VuZDoge1xyXG4gIFx0XHRpdGVtU3R5bGU6IHtcclxuICBcdFx0XHRmb250V2VpZ2h0OiAnYm9sZCcsXHJcbiAgXHRcdFx0Zm9udFNpemU6ICcxM3B4J1xyXG4gIFx0XHR9XHJcbiAgXHR9LFxyXG4gIFx0eEF4aXM6IHtcclxuICBcdFx0Z3JpZExpbmVXaWR0aDogMSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHR5QXhpczoge1xyXG4gIFx0XHRtaW5vclRpY2tJbnRlcnZhbDogJ2F1dG8nLFxyXG4gIFx0XHR0aXRsZToge1xyXG4gIFx0XHRcdHN0eWxlOiB7XHJcbiAgXHRcdFx0XHR0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xyXG4gIFx0XHRcdH1cclxuICBcdFx0fSxcclxuICBcdFx0bGFiZWxzOiB7XHJcbiAgXHRcdFx0c3R5bGU6IHtcclxuICBcdFx0XHRcdGZvbnRTaXplOiAnMTJweCdcclxuICBcdFx0XHR9XHJcbiAgXHRcdH1cclxuICBcdH0sXHJcbiAgXHRwbG90T3B0aW9ucz
 oge1xyXG4gIFx0XHRjYW5kbGVzdGljazoge1xyXG4gIFx0XHRcdGxpbmVDb2xvcjogJyM0MDQwNDgnXHJcbiAgXHRcdH1cclxuICBcdH0sXHJcblxyXG4gIFx0YmFja2dyb3VuZDI6ICcjRjBGMEVBJ1xyXG5cclxuICB9O1xyXG5cclxuICBIaWdoY2hhcnRzLnNldE9wdGlvbnMoSGlnaGNoYXJ0cy50aGVtZSk7XHJcblxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHVpVmlld1Njcm9sbFByb3ZpZGVyKSAtPlxyXG4gICR1aVZpZXdTY3JvbGxQcm92aWRlci51c2VBbmNob3JTY3JvbGwoKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbmZpZyAoJHN0YXRlUHJvdmlkZXIsICR1cmxSb3V0ZXJQcm92aWRlcikgLT5cclxuICAkc3RhdGVQcm92aWRlci5zdGF0ZSBcIm92ZXJ2aWV3XCIsXHJcbiAgICB1cmw6IFwiL292ZXJ2aWV3XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdPdmVydmlld0NvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInJ1bm5pbmctam9ic1wiLFxyXG4gICAgdXJsOiBcIi9ydW5uaW5nLWpvYnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgIG1haW46XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9ydW5uaW5
 nLWpvYnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcclxuICBcclxuICAuc3RhdGUgXCJjb21wbGV0ZWQtam9ic1wiLFxyXG4gICAgdXJsOiBcIi9jb21wbGV0ZWQtam9ic1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgbWFpbjpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdDb21wbGV0ZWRKb2JzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYlwiLFxyXG4gICAgdXJsOiBcIi9qb2JzL3tqb2JpZH1cIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVKb2JDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW5cIixcclxuICAgIHVybDogXCJcIlxyXG4gICAgYWJzdHJhY3Q6IHRydWVcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlBsYW5Db250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnBsYW4ub3ZlcnZpZXdcIixcclxuICAgIHVybDogXCJc
 IlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0Lm92ZXJ2aWV3Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJyBcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5wbGFuLmFjY3VtdWxhdG9yc1wiLFxyXG4gICAgdXJsOiBcIi9hY2N1bXVsYXRvcnNcIlxyXG4gICAgdmlld3M6XHJcbiAgICAgICdub2RlLWRldGFpbHMnOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4ubm9kZS1saXN0LmFjY3VtdWxhdG9ycy5odG1sXCJcclxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInIFxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lXCIsXHJcbiAgICB1cmw6IFwiL3RpbWVsaW5lXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnRpbWVsaW5lLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnRpbWVsaW5lLnZlcnRleFwiLFxyXG4gICAgdXJsOiBcIi97dmVydGV4SWR9XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICB2ZXJ0ZXg6XHJcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4L
 mh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2Iuc3RhdGlzdGljc1wiLFxyXG4gICAgdXJsOiBcIi9zdGF0aXN0aWNzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnN0YXRpc3RpY3MuaHRtbFwiXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1qb2IuZXhjZXB0aW9uc1wiLFxyXG4gICAgdXJsOiBcIi9leGNlcHRpb25zXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiXHJcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtam9iLnByb3BlcnRpZXNcIixcclxuICAgIHVybDogXCIvcHJvcGVydGllc1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wcm9wZXJ0aWVzLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwic2luZ2xlLWpvYi5jb25maWdcIixcclxuICAgIHVybDogXCIvY29uZmlnXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOl
 xyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmNvbmZpZy5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiYWxsLW1hbmFnZXJcIixcclxuICAgIHVybDogXCIvdGFza21hbmFnZXJzXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBtYWluOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL2luZGV4Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdBbGxUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlclwiLFxyXG4gICAgICB1cmw6IFwiL3Rhc2ttYW5hZ2VyL3t0YXNrbWFuYWdlcmlkfVwiXHJcbiAgICAgIHZpZXdzOlxyXG4gICAgICAgIG1haW46XHJcbiAgICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5odG1sXCJcclxuICAgICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVUYXNrTWFuYWdlckNvbnRyb2xsZXInXHJcblxyXG4gIC5zdGF0ZSBcInNpbmdsZS1tYW5hZ2VyLm1ldHJpY3NcIixcclxuICAgIHVybDogXCIvbWV0cmljc1wiXHJcbiAgICB2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5tZXRyaWNzLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlci5sb2dcIixcclxuICAgIHVybDogXCIvbG9nZmlsZVwiXHJcbiAgICB
 2aWV3czpcclxuICAgICAgZGV0YWlsczpcclxuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5sb2dmaWxlLmh0bWxcIlxyXG5cclxuICAuc3RhdGUgXCJzaW5nbGUtbWFuYWdlci5zdGRvdXRcIixcclxuICAgIHVybDogXCIvc3Rkb3V0XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN0ZG91dC5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiam9ibWFuYWdlclwiLFxyXG4gICAgICB1cmw6IFwiL2pvYm1hbmFnZXJcIlxyXG4gICAgICB2aWV3czpcclxuICAgICAgICBtYWluOlxyXG4gICAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9pbmRleC5odG1sXCJcclxuXHJcbiAgLnN0YXRlIFwiam9ibWFuYWdlci5jb25maWdcIixcclxuICAgIHVybDogXCIvY29uZmlnXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYm1hbmFnZXIvY29uZmlnLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JNYW5hZ2VyQ29uZmlnQ29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwiam9ibWFuYWdlci5zdGRvdXRcIixcclxuICAgIHVybDogXCIvc3Rkb3V0XCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRl
 VXJsOiBcInBhcnRpYWxzL2pvYm1hbmFnZXIvc3Rkb3V0Lmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JNYW5hZ2VyU3Rkb3V0Q29udHJvbGxlcidcclxuXHJcbiAgLnN0YXRlIFwiam9ibWFuYWdlci5sb2dcIixcclxuICAgIHVybDogXCIvbG9nXCJcclxuICAgIHZpZXdzOlxyXG4gICAgICBkZXRhaWxzOlxyXG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYm1hbmFnZXIvbG9nLmh0bWxcIlxyXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JNYW5hZ2VyTG9nc0NvbnRyb2xsZXInXHJcblxyXG4gICR1cmxSb3V0ZXJQcm92aWRlci5vdGhlcndpc2UgXCIvb3ZlcnZpZXdcIlxyXG5cclxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJywgWyd1aS5yb3V0ZXInLCAnYW5ndWxhck1vbWVudCddKS5ydW4oZnVuY3Rpb24oJHJvb3RTY29wZSkge1xuICAkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlID0gZmFsc2U7XG4gIHJldHVybiAkcm9vdFNjb3BlLnNob3dTaWRlYmFyID0gZnVuY3Rpb24oKSB7XG4gICAgJHJvb3RTY29wZS5zaWRlYmFyVmlzaWJsZSA9ICEkcm9vdFNjb3BlLnNpZGViYXJWaXNpYmxlO1xuICAgIHJldHVybiAkcm9vdFNjb3BlLnNpZGViYXJDbGFzcyA9ICdmb3JjZS1zaG93JztcbiAgfTtcbn0pLnZhbHVlKCdmbGlua0NvbmZpZycsIHtcbiAgXCJyZWZyZXNoLWludGVydmFsXCI6IDEwMDAwXG59KS5ydW4oZnVuY3Rpb24oSm9ic1NlcnZpY2UsIE1haW5TZXJ2aWNlLCBmbGlua
 0NvbmZpZywgJGludGVydmFsKSB7XG4gIE1haW5TZXJ2aWNlLmxvYWRDb25maWcoKS50aGVuKGZ1bmN0aW9uKGNvbmZpZykge1xuICAgIGFuZ3VsYXIuZXh0ZW5kKGZsaW5rQ29uZmlnLCBjb25maWcpO1xuICAgIEpvYnNTZXJ2aWNlLmxpc3RKb2JzKCk7XG4gICAgcmV0dXJuICRpbnRlcnZhbChmdW5jdGlvbigpIHtcbiAgICAgIHJldHVybiBKb2JzU2VydmljZS5saXN0Sm9icygpO1xuICAgIH0sIGZsaW5rQ29uZmlnW1wicmVmcmVzaC1pbnRlcnZhbFwiXSk7XG4gIH0pO1xuICBIaWdoY2hhcnRzLnNldE9wdGlvbnMoe1xuICAgIGdsb2JhbDoge1xuICAgICAgdXNlVVRDOiBmYWxzZVxuICAgIH1cbiAgfSk7XG4gIEhpZ2hjaGFydHMuY3JlYXRlRWxlbWVudCgnbGluaycsIHtcbiAgICBocmVmOiAnLy9mb250cy5nb29nbGVhcGlzLmNvbS9jc3M/ZmFtaWx5PURvc2lzOjQwMCw2MDAnLFxuICAgIHJlbDogJ3N0eWxlc2hlZXQnLFxuICAgIHR5cGU6ICd0ZXh0L2NzcydcbiAgfSwgbnVsbCwgZG9jdW1lbnQuZ2V0RWxlbWVudHNCeVRhZ05hbWUoJ2hlYWQnKVswXSk7XG4gIEhpZ2hjaGFydHMudGhlbWUgPSB7XG4gICAgY29sb3JzOiBbXCIjN2NiNWVjXCIsIFwiI2Y3YTM1Y1wiLCBcIiM5MGVlN2VcIiwgXCIjNzc5OEJGXCIsIFwiI2FhZWVlZVwiLCBcIiNmZjAwNjZcIiwgXCIjZWVhYWVlXCIsIFwiIzU1QkYzQlwiLCBcIiNERjUzNTNcIiwgXCIjNzc5OEJGXCIsIFwiI2FhZWVlZVwiXSxcbiAgICBjaGFydDoge1xuICAgIC
 AgYmFja2dyb3VuZENvbG9yOiBudWxsLFxuICAgICAgc3R5bGU6IHtcbiAgICAgICAgZm9udEZhbWlseTogXCJEb3Npcywgc2Fucy1zZXJpZlwiXG4gICAgICB9XG4gICAgfSxcbiAgICB0aXRsZToge1xuICAgICAgc3R5bGU6IHtcbiAgICAgICAgZm9udFNpemU6ICcxNnB4JyxcbiAgICAgICAgZm9udFdlaWdodDogJ2JvbGQnLFxuICAgICAgICB0ZXh0VHJhbnNmb3JtOiAndXBwZXJjYXNlJ1xuICAgICAgfVxuICAgIH0sXG4gICAgdG9vbHRpcDoge1xuICAgICAgYm9yZGVyV2lkdGg6IDAsXG4gICAgICBiYWNrZ3JvdW5kQ29sb3I6ICdyZ2JhKDIxOSwyMTksMjE2LDAuOCknLFxuICAgICAgc2hhZG93OiBmYWxzZVxuICAgIH0sXG4gICAgbGVnZW5kOiB7XG4gICAgICBpdGVtU3R5bGU6IHtcbiAgICAgICAgZm9udFdlaWdodDogJ2JvbGQnLFxuICAgICAgICBmb250U2l6ZTogJzEzcHgnXG4gICAgICB9XG4gICAgfSxcbiAgICB4QXhpczoge1xuICAgICAgZ3JpZExpbmVXaWR0aDogMSxcbiAgICAgIGxhYmVsczoge1xuICAgICAgICBzdHlsZToge1xuICAgICAgICAgIGZvbnRTaXplOiAnMTJweCdcbiAgICAgICAgfVxuICAgICAgfVxuICAgIH0sXG4gICAgeUF4aXM6IHtcbiAgICAgIG1pbm9yVGlja0ludGVydmFsOiAnYXV0bycsXG4gICAgICB0aXRsZToge1xuICAgICAgICBzdHlsZToge1xuICAgICAgICAgIHRleHRUcmFuc2Zvcm06ICd1cHBlcmNhc2UnXG4gICAgICAgIH1cbiAgICAgIH0sXG4gICAgICBsYWJlbHM
 6IHtcbiAgICAgICAgc3R5bGU6IHtcbiAgICAgICAgICBmb250U2l6ZTogJzEycHgnXG4gICAgICAgIH1cbiAgICAgIH1cbiAgICB9LFxuICAgIHBsb3RPcHRpb25zOiB7XG4gICAgICBjYW5kbGVzdGljazoge1xuICAgICAgICBsaW5lQ29sb3I6ICcjNDA0MDQ4J1xuICAgICAgfVxuICAgIH0sXG4gICAgYmFja2dyb3VuZDI6ICcjRjBGMEVBJ1xuICB9O1xuICByZXR1cm4gSGlnaGNoYXJ0cy5zZXRPcHRpb25zKEhpZ2hjaGFydHMudGhlbWUpO1xufSkuY29uZmlnKGZ1bmN0aW9uKCR1aVZpZXdTY3JvbGxQcm92aWRlcikge1xuICByZXR1cm4gJHVpVmlld1Njcm9sbFByb3ZpZGVyLnVzZUFuY2hvclNjcm9sbCgpO1xufSkuY29uZmlnKGZ1bmN0aW9uKCRzdGF0ZVByb3ZpZGVyLCAkdXJsUm91dGVyUHJvdmlkZXIpIHtcbiAgJHN0YXRlUHJvdmlkZXIuc3RhdGUoXCJvdmVydmlld1wiLCB7XG4gICAgdXJsOiBcIi9vdmVydmlld1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL292ZXJ2aWV3Lmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ092ZXJ2aWV3Q29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwicnVubmluZy1qb2JzXCIsIHtcbiAgICB1cmw6IFwiL3J1bm5pbmctam9ic1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvcnVubmluZy1qb2Jz
 Lmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ1J1bm5pbmdKb2JzQ29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwiY29tcGxldGVkLWpvYnNcIiwge1xuICAgIHVybDogXCIvY29tcGxldGVkLWpvYnNcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2NvbXBsZXRlZC1qb2JzLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0NvbXBsZXRlZEpvYnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iXCIsIHtcbiAgICB1cmw6IFwiL2pvYnMve2pvYmlkfVwiLFxuICAgIGFic3RyYWN0OiB0cnVlLFxuICAgIHZpZXdzOiB7XG4gICAgICBtYWluOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ1NpbmdsZUpvYkNvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IucGxhblwiLCB7XG4gICAgdXJsOiBcIlwiLFxuICAgIGFic3RyYWN0OiB0cnVlLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLnBsYW4uaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkNvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2Iuc
 Gxhbi5vdmVydmlld1wiLCB7XG4gICAgdXJsOiBcIlwiLFxuICAgIHZpZXdzOiB7XG4gICAgICAnbm9kZS1kZXRhaWxzJzoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLm5vZGUtbGlzdC5vdmVydmlldy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQbGFuT3ZlcnZpZXdDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLnBsYW4uYWNjdW11bGF0b3JzXCIsIHtcbiAgICB1cmw6IFwiL2FjY3VtdWxhdG9yc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICAnbm9kZS1kZXRhaWxzJzoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5wbGFuLm5vZGUtbGlzdC5hY2N1bXVsYXRvcnMuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IudGltZWxpbmVcIiwge1xuICAgIHVybDogXCIvdGltZWxpbmVcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi50aW1lbGluZS5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi50aW1lbGluZS52ZXJ0ZXhcIiwge1xuICAgIHVybDogXCIve3ZlcnRleElkfVwiLFxuICAgIHZpZXdzOiB7XG4gICAgICB2ZXJ0ZXg6IH
 tcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IudGltZWxpbmUudmVydGV4Lmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcidcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5zdGF0aXN0aWNzXCIsIHtcbiAgICB1cmw6IFwiL3N0YXRpc3RpY3NcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JzL2pvYi5zdGF0aXN0aWNzLmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtam9iLmV4Y2VwdGlvbnNcIiwge1xuICAgIHVybDogXCIvZXhjZXB0aW9uc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYnMvam9iLmV4Y2VwdGlvbnMuaHRtbFwiLFxuICAgICAgICBjb250cm9sbGVyOiAnSm9iRXhjZXB0aW9uc0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1qb2IucHJvcGVydGllc1wiLCB7XG4gICAgdXJsOiBcIi9wcm9wZXJ0aWVzXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IucHJvcGVydGllcy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcbiAgICA
 gIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLWpvYi5jb25maWdcIiwge1xuICAgIHVybDogXCIvY29uZmlnXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9icy9qb2IuY29uZmlnLmh0bWxcIlxuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJhbGwtbWFuYWdlclwiLCB7XG4gICAgdXJsOiBcIi90YXNrbWFuYWdlcnNcIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci9pbmRleC5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdBbGxUYXNrTWFuYWdlcnNDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJzaW5nbGUtbWFuYWdlclwiLCB7XG4gICAgdXJsOiBcIi90YXNrbWFuYWdlci97dGFza21hbmFnZXJpZH1cIixcbiAgICB2aWV3czoge1xuICAgICAgbWFpbjoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy90YXNrbWFuYWdlci90YXNrbWFuYWdlci5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdTaW5nbGVUYXNrTWFuYWdlckNvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1tYW5hZ2VyLm1ldHJpY3NcIiwge1xuICAgIHVybDogXCIvbWV0cmljc1wiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBc
 InBhcnRpYWxzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLm1ldHJpY3MuaHRtbFwiXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcInNpbmdsZS1tYW5hZ2VyLmxvZ1wiLCB7XG4gICAgdXJsOiBcIi9sb2dmaWxlXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvdGFza21hbmFnZXIvdGFza21hbmFnZXIubG9nZmlsZS5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwic2luZ2xlLW1hbmFnZXIuc3Rkb3V0XCIsIHtcbiAgICB1cmw6IFwiL3N0ZG91dFwiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL3Rhc2ttYW5hZ2VyL3Rhc2ttYW5hZ2VyLnN0ZG91dC5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwiam9ibWFuYWdlclwiLCB7XG4gICAgdXJsOiBcIi9qb2JtYW5hZ2VyXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIG1haW46IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9pbmRleC5odG1sXCJcbiAgICAgIH1cbiAgICB9XG4gIH0pLnN0YXRlKFwiam9ibWFuYWdlci5jb25maWdcIiwge1xuICAgIHVybDogXCIvY29uZmlnXCIsXG4gICAgdmlld3M6IHtcbiAgICAgIGRldGFpbHM6IHtcbiAgICAgICAgdGVtcGxhdGVVcmw6IFwicGFydGlhbHMvam9ibWFuYWdlci9jb25maWcuaHRtbFwiLFxuICAgICAgI
 CBjb250cm9sbGVyOiAnSm9iTWFuYWdlckNvbmZpZ0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KS5zdGF0ZShcImpvYm1hbmFnZXIuc3Rkb3V0XCIsIHtcbiAgICB1cmw6IFwiL3N0ZG91dFwiLFxuICAgIHZpZXdzOiB7XG4gICAgICBkZXRhaWxzOiB7XG4gICAgICAgIHRlbXBsYXRlVXJsOiBcInBhcnRpYWxzL2pvYm1hbmFnZXIvc3Rkb3V0Lmh0bWxcIixcbiAgICAgICAgY29udHJvbGxlcjogJ0pvYk1hbmFnZXJTdGRvdXRDb250cm9sbGVyJ1xuICAgICAgfVxuICAgIH1cbiAgfSkuc3RhdGUoXCJqb2JtYW5hZ2VyLmxvZ1wiLCB7XG4gICAgdXJsOiBcIi9sb2dcIixcbiAgICB2aWV3czoge1xuICAgICAgZGV0YWlsczoge1xuICAgICAgICB0ZW1wbGF0ZVVybDogXCJwYXJ0aWFscy9qb2JtYW5hZ2VyL2xvZy5odG1sXCIsXG4gICAgICAgIGNvbnRyb2xsZXI6ICdKb2JNYW5hZ2VyTG9nc0NvbnRyb2xsZXInXG4gICAgICB9XG4gICAgfVxuICB9KTtcbiAgcmV0dXJuICR1cmxSb3V0ZXJQcm92aWRlci5vdGhlcndpc2UoXCIvb3ZlcnZpZXdcIik7XG59KTtcbiIsIiNcclxuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXHJcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXHJcbiMgZGlzdHJpYnV0ZWQgd2l0aCB0aGlzIHdvcmsgZm9yIGFkZGl0aW9uYWwgaW5mb3JtYXRpb25cclxuIy
 ByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXHJcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxyXG4jIFwiTGljZW5zZVwiKTsgeW91IG1heSBub3QgdXNlIHRoaXMgZmlsZSBleGNlcHQgaW4gY29tcGxpYW5jZVxyXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcclxuI1xyXG4jICAgICBodHRwOi8vd3d3LmFwYWNoZS5vcmcvbGljZW5zZXMvTElDRU5TRS0yLjBcclxuI1xyXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcclxuIyBkaXN0cmlidXRlZCB1bmRlciB0aGUgTGljZW5zZSBpcyBkaXN0cmlidXRlZCBvbiBhbiBcIkFTIElTXCIgQkFTSVMsXHJcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXHJcbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxyXG4jIGxpbWl0YXRpb25zIHVuZGVyIHRoZSBMaWNlbnNlLlxyXG4jXHJcblxyXG5hbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4uZGlyZWN
 0aXZlICdic0xhYmVsJywgKEpvYnNTZXJ2aWNlKSAtPlxyXG4gIHRyYW5zY2x1ZGU6IHRydWVcclxuICByZXBsYWNlOiB0cnVlXHJcbiAgc2NvcGU6IFxyXG4gICAgZ2V0TGFiZWxDbGFzczogXCImXCJcclxuICAgIHN0YXR1czogXCJAXCJcclxuXHJcbiAgdGVtcGxhdGU6IFwiPHNwYW4gdGl0bGU9J3t7c3RhdHVzfX0nIG5nLWNsYXNzPSdnZXRMYWJlbENsYXNzKCknPjxuZy10cmFuc2NsdWRlPjwvbmctdHJhbnNjbHVkZT48L3NwYW4+XCJcclxuICBcclxuICBsaW5rOiAoc2NvcGUsIGVsZW1lbnQsIGF0dHJzKSAtPlxyXG4gICAgc2NvcGUuZ2V0TGFiZWxDbGFzcyA9IC0+XHJcbiAgICAgICdsYWJlbCBsYWJlbC0nICsgSm9ic1NlcnZpY2UudHJhbnNsYXRlTGFiZWxTdGF0ZShhdHRycy5zdGF0dXMpXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5kaXJlY3RpdmUgJ2luZGljYXRvclByaW1hcnknLCAoSm9ic1NlcnZpY2UpIC0+XHJcbiAgcmVwbGFjZTogdHJ1ZVxyXG4gIHNjb3BlOiBcclxuICAgIGdldExhYmVsQ2xhc3M6IFwiJlwiXHJcbiAgICBzdGF0dXM6ICdAJ1xyXG5cclxuICB0ZW1wbGF0ZTogXCI8aSB0aXRsZT0ne3tzdGF0dXN9fScgbmctY2xhc3M9J2dldExhYmVsQ2xhc3MoKScgLz5cIlxyXG4gIFxyXG4gIGxpbms6IChzY29wZSwgZWxlbWVudCwgYXR0cnMpIC0+XHJcbiAgICBzY29wZS5nZXRMYWJlbENsYXNzID0gLT5cclxuICAgICAg
 J2ZhIGZhLWNpcmNsZSBpbmRpY2F0b3IgaW5kaWNhdG9yLScgKyBKb2JzU2VydmljZS50cmFuc2xhdGVMYWJlbFN0YXRlKGF0dHJzLnN0YXR1cylcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmRpcmVjdGl2ZSAndGFibGVQcm9wZXJ0eScsIC0+XHJcbiAgcmVwbGFjZTogdHJ1ZVxyXG4gIHNjb3BlOlxyXG4gICAgdmFsdWU6ICc9J1xyXG5cclxuICB0ZW1wbGF0ZTogXCI8dGQgdGl0bGU9XFxcInt7dmFsdWUgfHwgJ05vbmUnfX1cXFwiPnt7dmFsdWUgfHwgJ05vbmUnfX08L3RkPlwiXHJcbiIsImFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpLmRpcmVjdGl2ZSgnYnNMYWJlbCcsIGZ1bmN0aW9uKEpvYnNTZXJ2aWNlKSB7XG4gIHJldHVybiB7XG4gICAgdHJhbnNjbHVkZTogdHJ1ZSxcbiAgICByZXBsYWNlOiB0cnVlLFxuICAgIHNjb3BlOiB7XG4gICAgICBnZXRMYWJlbENsYXNzOiBcIiZcIixcbiAgICAgIHN0YXR1czogXCJAXCJcbiAgICB9LFxuICAgIHRlbXBsYXRlOiBcIjxzcGFuIHRpdGxlPSd7e3N0YXR1c319JyBuZy1jbGFzcz0nZ2V0TGFiZWxDbGFzcygpJz48bmctdHJhbnNjbHVkZT48L25nLXRyYW5zY2x1ZGU+PC9zcGFuPlwiLFxuICAgIGxpbms6IGZ1bmN0aW9uKHNjb3BlLCBlbGVtZW50LCBhdHRycykge1xuICAgICAgcmV0dXJuIHNjb3BlLmdldExhYmVsQ2xhc3MgPSBmdW5jdGlvbigpIHtcbiAgICAgICAgcmV0dXJuICdsYWJlbCBsY
 WJlbC0nICsgSm9ic1NlcnZpY2UudHJhbnNsYXRlTGFiZWxTdGF0ZShhdHRycy5zdGF0dXMpO1xuICAgICAgfTtcbiAgICB9XG4gIH07XG59KS5kaXJlY3RpdmUoJ2luZGljYXRvclByaW1hcnknLCBmdW5jdGlvbihKb2JzU2VydmljZSkge1xuICByZXR1cm4ge1xuICAgIHJlcGxhY2U6IHRydWUsXG4gICAgc2NvcGU6IHtcbiAgICAgIGdldExhYmVsQ2xhc3M6IFwiJlwiLFxuICAgICAgc3RhdHVzOiAnQCdcbiAgICB9LFxuICAgIHRlbXBsYXRlOiBcIjxpIHRpdGxlPSd7e3N0YXR1c319JyBuZy1jbGFzcz0nZ2V0TGFiZWxDbGFzcygpJyAvPlwiLFxuICAgIGxpbms6IGZ1bmN0aW9uKHNjb3BlLCBlbGVtZW50LCBhdHRycykge1xuICAgICAgcmV0dXJuIHNjb3BlLmdldExhYmVsQ2xhc3MgPSBmdW5jdGlvbigpIHtcbiAgICAgICAgcmV0dXJuICdmYSBmYS1jaXJjbGUgaW5kaWNhdG9yIGluZGljYXRvci0nICsgSm9ic1NlcnZpY2UudHJhbnNsYXRlTGFiZWxTdGF0ZShhdHRycy5zdGF0dXMpO1xuICAgICAgfTtcbiAgICB9XG4gIH07XG59KS5kaXJlY3RpdmUoJ3RhYmxlUHJvcGVydHknLCBmdW5jdGlvbigpIHtcbiAgcmV0dXJuIHtcbiAgICByZXBsYWNlOiB0cnVlLFxuICAgIHNjb3BlOiB7XG4gICAgICB2YWx1ZTogJz0nXG4gICAgfSxcbiAgICB0ZW1wbGF0ZTogXCI8dGQgdGl0bGU9XFxcInt7dmFsdWUgfHwgJ05vbmUnfX1cXFwiPnt7dmFsdWUgfHwgJ05vbmUnfX08L3RkPlwiXG4gIH07XG59KTtcbiIsIiNcclxuIy
 BMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXHJcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXHJcbiMgZGlzdHJpYnV0ZWQgd2l0aCB0aGlzIHdvcmsgZm9yIGFkZGl0aW9uYWwgaW5mb3JtYXRpb25cclxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXHJcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxyXG4jIFwiTGljZW5zZVwiKTsgeW91IG1heSBub3QgdXNlIHRoaXMgZmlsZSBleGNlcHQgaW4gY29tcGxpYW5jZVxyXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcclxuI1xyXG4jICAgICBodHRwOi8vd3d3LmFwYWNoZS5vcmcvbGljZW5zZXMvTElDRU5TRS0yLjBcclxuI1xyXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcclxuIyBkaXN0cmlidXRlZCB1bmRlciB0aGUgTGljZW5zZSBpcyBkaXN0cmlidXRlZCBvbiBhbiBcIkFTIElTXCIgQkFTSVMsXHJcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXHJcbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGU
 gc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxyXG4jIGxpbWl0YXRpb25zIHVuZGVyIHRoZSBMaWNlbnNlLlxyXG4jXHJcblxyXG5hbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKVxyXG5cclxuLmZpbHRlciBcImFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZFwiLCAoYW5ndWxhck1vbWVudENvbmZpZykgLT5cclxuICBhbUR1cmF0aW9uRm9ybWF0RXh0ZW5kZWRGaWx0ZXIgPSAodmFsdWUsIGZvcm1hdCwgZHVyYXRpb25Gb3JtYXQpIC0+XHJcbiAgICByZXR1cm4gXCJcIiAgaWYgdHlwZW9mIHZhbHVlIGlzIFwidW5kZWZpbmVkXCIgb3IgdmFsdWUgaXMgbnVsbFxyXG5cclxuICAgIG1vbWVudC5kdXJhdGlvbih2YWx1ZSwgZm9ybWF0KS5mb3JtYXQoZHVyYXRpb25Gb3JtYXQsIHsgdHJpbTogZmFsc2UgfSlcclxuXHJcbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyLiRzdGF0ZWZ1bCA9IGFuZ3VsYXJNb21lbnRDb25maWcuc3RhdGVmdWxGaWx0ZXJzXHJcblxyXG4gIGFtRHVyYXRpb25Gb3JtYXRFeHRlbmRlZEZpbHRlclxyXG5cclxuLmZpbHRlciBcImh1bWFuaXplVGV4dFwiLCAtPlxyXG4gICh0ZXh0KSAtPlxyXG4gICAgIyBUT0RPOiBleHRlbmQuLi4gYSBsb3RcclxuICAgIGlmIHRleHQgdGhlbiB0ZXh0LnJlcGxhY2UoLyZndDsvZywgXCI+XCIpLnJlcGxhY2UoLzxiclxcLz4vZyxcIlwiKSBlbHNlICcnXHJcblxyXG4uZmlsdGVyIFwiYnl0ZXNcIiwgLT5cclxuICAoYnl0ZXMs
 IHByZWNpc2lvbikgLT5cclxuICAgIHJldHVybiBcIi1cIiAgaWYgaXNOYU4ocGFyc2VGbG9hdChieXRlcykpIG9yIG5vdCBpc0Zpbml0ZShieXRlcylcclxuICAgIHByZWNpc2lvbiA9IDEgIGlmIHR5cGVvZiBwcmVjaXNpb24gaXMgXCJ1bmRlZmluZWRcIlxyXG4gICAgdW5pdHMgPSBbIFwiYnl0ZXNcIiwgXCJrQlwiLCBcIk1CXCIsIFwiR0JcIiwgXCJUQlwiLCBcIlBCXCIgXVxyXG4gICAgbnVtYmVyID0gTWF0aC5mbG9vcihNYXRoLmxvZyhieXRlcykgLyBNYXRoLmxvZygxMDI0KSlcclxuICAgIChieXRlcyAvIE1hdGgucG93KDEwMjQsIE1hdGguZmxvb3IobnVtYmVyKSkpLnRvRml4ZWQocHJlY2lzaW9uKSArIFwiIFwiICsgdW5pdHNbbnVtYmVyXVxyXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5maWx0ZXIoXCJhbUR1cmF0aW9uRm9ybWF0RXh0ZW5kZWRcIiwgZnVuY3Rpb24oYW5ndWxhck1vbWVudENvbmZpZykge1xuICB2YXIgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyO1xuICBhbUR1cmF0aW9uRm9ybWF0RXh0ZW5kZWRGaWx0ZXIgPSBmdW5jdGlvbih2YWx1ZSwgZm9ybWF0LCBkdXJhdGlvbkZvcm1hdCkge1xuICAgIGlmICh0eXBlb2YgdmFsdWUgPT09IFwidW5kZWZpbmVkXCIgfHwgdmFsdWUgPT09IG51bGwpIHtcbiAgICAgIHJldHVybiBcIlwiO1xuICAgIH1cbiAgICByZXR1cm4gbW9tZW50LmR1cmF0aW9uKHZhbHVlLCBmb3JtYXQpLmZvcm1hdChkdXJhdGlvbkZvcm1hdCwge1xuI
 CAgICAgdHJpbTogZmFsc2VcbiAgICB9KTtcbiAgfTtcbiAgYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyLiRzdGF0ZWZ1bCA9IGFuZ3VsYXJNb21lbnRDb25maWcuc3RhdGVmdWxGaWx0ZXJzO1xuICByZXR1cm4gYW1EdXJhdGlvbkZvcm1hdEV4dGVuZGVkRmlsdGVyO1xufSkuZmlsdGVyKFwiaHVtYW5pemVUZXh0XCIsIGZ1bmN0aW9uKCkge1xuICByZXR1cm4gZnVuY3Rpb24odGV4dCkge1xuICAgIGlmICh0ZXh0KSB7XG4gICAgICByZXR1cm4gdGV4dC5yZXBsYWNlKC8mZ3Q7L2csIFwiPlwiKS5yZXBsYWNlKC88YnJcXC8+L2csIFwiXCIpO1xuICAgIH0gZWxzZSB7XG4gICAgICByZXR1cm4gJyc7XG4gICAgfVxuICB9O1xufSkuZmlsdGVyKFwiYnl0ZXNcIiwgZnVuY3Rpb24oKSB7XG4gIHJldHVybiBmdW5jdGlvbihieXRlcywgcHJlY2lzaW9uKSB7XG4gICAgdmFyIG51bWJlciwgdW5pdHM7XG4gICAgaWYgKGlzTmFOKHBhcnNlRmxvYXQoYnl0ZXMpKSB8fCAhaXNGaW5pdGUoYnl0ZXMpKSB7XG4gICAgICByZXR1cm4gXCItXCI7XG4gICAgfVxuICAgIGlmICh0eXBlb2YgcHJlY2lzaW9uID09PSBcInVuZGVmaW5lZFwiKSB7XG4gICAgICBwcmVjaXNpb24gPSAxO1xuICAgIH1cbiAgICB1bml0cyA9IFtcImJ5dGVzXCIsIFwia0JcIiwgXCJNQlwiLCBcIkdCXCIsIFwiVEJcIiwgXCJQQlwiXTtcbiAgICBudW1iZXIgPSBNYXRoLmZsb29yKE1hdGgubG9nKGJ5dGVzKSAvIE1hdGgubG9nKDEwMjQpKTtcbiAgIC
 ByZXR1cm4gKGJ5dGVzIC8gTWF0aC5wb3coMTAyNCwgTWF0aC5mbG9vcihudW1iZXIpKSkudG9GaXhlZChwcmVjaXNpb24pICsgXCIgXCIgKyB1bml0c1tudW1iZXJdO1xuICB9O1xufSk7XG4iLCIjXHJcbiMgTGljZW5zZWQgdG8gdGhlIEFwYWNoZSBTb2Z0d2FyZSBGb3VuZGF0aW9uIChBU0YpIHVuZGVyIG9uZVxyXG4jIG9yIG1vcmUgY29udHJpYnV0b3IgbGljZW5zZSBhZ3JlZW1lbnRzLiAgU2VlIHRoZSBOT1RJQ0UgZmlsZVxyXG4jIGRpc3RyaWJ1dGVkIHdpdGggdGhpcyB3b3JrIGZvciBhZGRpdGlvbmFsIGluZm9ybWF0aW9uXHJcbiMgcmVnYXJkaW5nIGNvcHlyaWdodCBvd25lcnNoaXAuICBUaGUgQVNGIGxpY2Vuc2VzIHRoaXMgZmlsZVxyXG4jIHRvIHlvdSB1bmRlciB0aGUgQXBhY2hlIExpY2Vuc2UsIFZlcnNpb24gMi4wICh0aGVcclxuIyBcIkxpY2Vuc2VcIik7IHlvdSBtYXkgbm90IHVzZSB0aGlzIGZpbGUgZXhjZXB0IGluIGNvbXBsaWFuY2VcclxuIyB3aXRoIHRoZSBMaWNlbnNlLiAgWW91IG1heSBvYnRhaW4gYSBjb3B5IG9mIHRoZSBMaWNlbnNlIGF0XHJcbiNcclxuIyAgICAgaHR0cDovL3d3dy5hcGFjaGUub3JnL2xpY2Vuc2VzL0xJQ0VOU0UtMi4wXHJcbiNcclxuIyBVbmxlc3MgcmVxdWlyZWQgYnkgYXBwbGljYWJsZSBsYXcgb3IgYWdyZWVkIHRvIGluIHdyaXRpbmcsIHNvZnR3YXJlXHJcbiMgZGlzdHJpYnV0ZWQgdW5kZXIgdGhlIExpY2Vuc2UgaXMgZGlzdHJpYnV0ZWQgb24gYW4gXCJBUyB
 JU1wiIEJBU0lTLFxyXG4jIFdJVEhPVVQgV0FSUkFOVElFUyBPUiBDT05ESVRJT05TIE9GIEFOWSBLSU5ELCBlaXRoZXIgZXhwcmVzcyBvciBpbXBsaWVkLlxyXG4jIFNlZSB0aGUgTGljZW5zZSBmb3IgdGhlIHNwZWNpZmljIGxhbmd1YWdlIGdvdmVybmluZyBwZXJtaXNzaW9ucyBhbmRcclxuIyBsaW1pdGF0aW9ucyB1bmRlciB0aGUgTGljZW5zZS5cclxuI1xyXG5cclxuYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJylcclxuXHJcbi5zZXJ2aWNlICdNYWluU2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRxKSAtPlxyXG4gIEBsb2FkQ29uZmlnID0gLT5cclxuICAgIGRlZmVycmVkID0gJHEuZGVmZXIoKVxyXG5cclxuICAgICRodHRwLmdldCBcImNvbmZpZ1wiXHJcbiAgICAuc3VjY2VzcyAoZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIC0+XHJcbiAgICAgIGRlZmVycmVkLnJlc29sdmUoZGF0YSlcclxuXHJcbiAgICBkZWZlcnJlZC5wcm9taXNlXHJcblxyXG5cclxuICBAXHJcbiIsImFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpLnNlcnZpY2UoJ01haW5TZXJ2aWNlJywgZnVuY3Rpb24oJGh0dHAsIGZsaW5rQ29uZmlnLCAkcSkge1xuICB0aGlzLmxvYWRDb25maWcgPSBmdW5jdGlvbigpIHtcbiAgICB2YXIgZGVmZXJyZWQ7XG4gICAgZGVmZXJyZWQgPSAkcS5kZWZlcigpO1xuICAgICRodHRwLmdldChcImNvbmZpZ1wiKS5zdWNjZXNzKGZ1bmN0aW9uKGRhdGEsIHN0YXR1cywgaGVhZGVycywgY29u
 ZmlnKSB7XG4gICAgICByZXR1cm4gZGVmZXJyZWQucmVzb2x2ZShkYXRhKTtcbiAgICB9KTtcbiAgICByZXR1cm4gZGVmZXJyZWQucHJvbWlzZTtcbiAgfTtcbiAgcmV0dXJuIHRoaXM7XG59KTtcbiIsIiNcclxuIyBMaWNlbnNlZCB0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXHJcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXHJcbiMgZGlzdHJpYnV0ZWQgd2l0aCB0aGlzIHdvcmsgZm9yIGFkZGl0aW9uYWwgaW5mb3JtYXRpb25cclxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXHJcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxyXG4jIFwiTGljZW5zZVwiKTsgeW91IG1heSBub3QgdXNlIHRoaXMgZmlsZSBleGNlcHQgaW4gY29tcGxpYW5jZVxyXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcclxuI1xyXG4jICAgICBodHRwOi8vd3d3LmFwYWNoZS5vcmcvbGljZW5zZXMvTElDRU5TRS0yLjBcclxuI1xyXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcclxuIyBkaXN0cmlidXRlZCB1bmRlciB0aGUgTGljZW5zZSBpcyBkaXN0cmlidXRlZCBvbiBhbiBcI
 kFTIElTXCIgQkFTSVMsXHJcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXHJcbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMgbGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxyXG4jIGxpbWl0YXRpb25zIHVuZGVyIHRoZSBMaWNlbnNlLlxyXG4jXHJcblxyXG5hbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKVxyXG5cclxuLmNvbnRyb2xsZXIgJ0pvYk1hbmFnZXJDb25maWdDb250cm9sbGVyJywgKCRzY29wZSwgSm9iTWFuYWdlckNvbmZpZ1NlcnZpY2UpIC0+XHJcbiAgSm9iTWFuYWdlckNvbmZpZ1NlcnZpY2UubG9hZENvbmZpZygpLnRoZW4gKGRhdGEpIC0+XHJcbiAgICBpZiAhJHNjb3BlLmpvYm1hbmFnZXI/XHJcbiAgICAgICRzY29wZS5qb2JtYW5hZ2VyID0ge31cclxuICAgICRzY29wZS5qb2JtYW5hZ2VyWydjb25maWcnXSA9IGRhdGFcclxuXHJcbi5jb250cm9sbGVyICdKb2JNYW5hZ2VyTG9nc0NvbnRyb2xsZXInLCAoJHNjb3BlLCBKb2JNYW5hZ2VyTG9nc1NlcnZpY2UpIC0+XHJcbiAgSm9iTWFuYWdlckxvZ3NTZXJ2aWNlLmxvYWRMb2dzKCkudGhlbiAoZGF0YSkgLT5cclxuICAgIGlmICEkc2NvcGUuam9ibWFuYWdlcj9cclxuICAgICAgJHNjb3BlLmpvYm1hbmFnZXIgPSB7fVxyXG4gICAgJHNjb3BlLmpvYm1hbmFnZXJbJ2xvZyddID0gZGF0YVxyXG5cclxuICAkc2NvcGUucmVsb2FkRG
 F0YSA9ICgpIC0+XHJcbiAgICBKb2JNYW5hZ2VyTG9nc1NlcnZpY2UubG9hZExvZ3MoKS50aGVuIChkYXRhKSAtPlxyXG4gICAgICAkc2NvcGUuam9ibWFuYWdlclsnbG9nJ10gPSBkYXRhXHJcblxyXG4uY29udHJvbGxlciAnSm9iTWFuYWdlclN0ZG91dENvbnRyb2xsZXInLCAoJHNjb3BlLCBKb2JNYW5hZ2VyU3Rkb3V0U2VydmljZSkgLT5cclxuICBKb2JNYW5hZ2VyU3Rkb3V0U2VydmljZS5sb2FkU3Rkb3V0KCkudGhlbiAoZGF0YSkgLT5cclxuICAgIGlmICEkc2NvcGUuam9ibWFuYWdlcj9cclxuICAgICAgJHNjb3BlLmpvYm1hbmFnZXIgPSB7fVxyXG4gICAgJHNjb3BlLmpvYm1hbmFnZXJbJ3N0ZG91dCddID0gZGF0YVxyXG5cclxuICAkc2NvcGUucmVsb2FkRGF0YSA9ICgpIC0+XHJcbiAgICBKb2JNYW5hZ2VyU3Rkb3V0U2VydmljZS5sb2FkU3Rkb3V0KCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgJHNjb3BlLmpvYm1hbmFnZXJbJ3N0ZG91dCddID0gZGF0YVxyXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5jb250cm9sbGVyKCdKb2JNYW5hZ2VyQ29uZmlnQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgSm9iTWFuYWdlckNvbmZpZ1NlcnZpY2UpIHtcbiAgcmV0dXJuIEpvYk1hbmFnZXJDb25maWdTZXJ2aWNlLmxvYWRDb25maWcoKS50aGVuKGZ1bmN0aW9uKGRhdGEpIHtcbiAgICBpZiAoJHNjb3BlLmpvYm1hbmFnZXIgPT0gbnVsbCkge1xuICAgICAgJHNjb3BlLmpvYm1hbmFnZXIgPSB7fTt
 cbiAgICB9XG4gICAgcmV0dXJuICRzY29wZS5qb2JtYW5hZ2VyWydjb25maWcnXSA9IGRhdGE7XG4gIH0pO1xufSkuY29udHJvbGxlcignSm9iTWFuYWdlckxvZ3NDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCBKb2JNYW5hZ2VyTG9nc1NlcnZpY2UpIHtcbiAgSm9iTWFuYWdlckxvZ3NTZXJ2aWNlLmxvYWRMb2dzKCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgaWYgKCRzY29wZS5qb2JtYW5hZ2VyID09IG51bGwpIHtcbiAgICAgICRzY29wZS5qb2JtYW5hZ2VyID0ge307XG4gICAgfVxuICAgIHJldHVybiAkc2NvcGUuam9ibWFuYWdlclsnbG9nJ10gPSBkYXRhO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5yZWxvYWREYXRhID0gZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYk1hbmFnZXJMb2dzU2VydmljZS5sb2FkTG9ncygpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICAgcmV0dXJuICRzY29wZS5qb2JtYW5hZ2VyWydsb2cnXSA9IGRhdGE7XG4gICAgfSk7XG4gIH07XG59KS5jb250cm9sbGVyKCdKb2JNYW5hZ2VyU3Rkb3V0Q29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgSm9iTWFuYWdlclN0ZG91dFNlcnZpY2UpIHtcbiAgSm9iTWFuYWdlclN0ZG91dFNlcnZpY2UubG9hZFN0ZG91dCgpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgIGlmICgkc2NvcGUuam9ibWFuYWdlciA9PSBudWxsKSB7XG4gICAgICAkc2NvcGUuam9ibWFuYWdlciA9IHt9O1xuICAgIH1cbiAgICByZXR1cm4gJHNj
 b3BlLmpvYm1hbmFnZXJbJ3N0ZG91dCddID0gZGF0YTtcbiAgfSk7XG4gIHJldHVybiAkc2NvcGUucmVsb2FkRGF0YSA9IGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiBKb2JNYW5hZ2VyU3Rkb3V0U2VydmljZS5sb2FkU3Rkb3V0KCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgICByZXR1cm4gJHNjb3BlLmpvYm1hbmFnZXJbJ3N0ZG91dCddID0gZGF0YTtcbiAgICB9KTtcbiAgfTtcbn0pO1xuIiwiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyXG4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpc
 mVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXHJcblxyXG4uc2VydmljZSAnSm9iTWFuYWdlckNvbmZpZ1NlcnZpY2UnLCAoJGh0dHAsIGZsaW5rQ29uZmlnLCAkcSkgLT5cclxuICBjb25maWcgPSB7fVxyXG5cclxuICBAbG9hZENvbmZpZyA9IC0+XHJcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKClcclxuXHJcbiAgICAkaHR0cC5nZXQoXCJqb2JtYW5hZ2VyL2NvbmZpZ1wiKVxyXG4gICAgLnN1Y2Nlc3MgKGRhdGEsIHN0YXR1cywgaGVhZGVycywgY29uZmlnKSAtPlxyXG4gICAgICBjb25maWcgPSBkYXRhXHJcbiAgICAgIGRlZmVycmVkLnJlc29sdmUoZGF0YSlcclxuXHJcbiAgICBkZWZlcnJlZC5wcm9taXNlXHJcblxyXG4gIEBcclxuXHJcbi5zZXJ2aWNlICdKb2JNYW5hZ2VyTG9nc1NlcnZpY2UnLCAoJGh0dHAsIGZsaW5rQ29uZmlnLC
 AkcSkgLT5cclxuICBsb2dzID0ge31cclxuXHJcbiAgQGxvYWRMb2dzID0gLT5cclxuICAgIGRlZmVycmVkID0gJHEuZGVmZXIoKVxyXG5cclxuICAgICRodHRwLmdldChcImpvYm1hbmFnZXIvbG9nXCIpXHJcbiAgICAuc3VjY2VzcyAoZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIC0+XHJcbiAgICAgIGxvZ3MgPSBkYXRhXHJcbiAgICAgIGRlZmVycmVkLnJlc29sdmUoZGF0YSlcclxuXHJcbiAgICBkZWZlcnJlZC5wcm9taXNlXHJcblxyXG4gIEBcclxuXHJcbi5zZXJ2aWNlICdKb2JNYW5hZ2VyU3Rkb3V0U2VydmljZScsICgkaHR0cCwgZmxpbmtDb25maWcsICRxKSAtPlxyXG4gIHN0ZG91dCA9IHt9XHJcblxyXG4gIEBsb2FkU3Rkb3V0ID0gLT5cclxuICAgIGRlZmVycmVkID0gJHEuZGVmZXIoKVxyXG5cclxuICAgICRodHRwLmdldChcImpvYm1hbmFnZXIvc3Rkb3V0XCIpXHJcbiAgICAuc3VjY2VzcyAoZGF0YSwgc3RhdHVzLCBoZWFkZXJzLCBjb25maWcpIC0+XHJcbiAgICAgIHN0ZG91dCA9IGRhdGFcclxuICAgICAgZGVmZXJyZWQucmVzb2x2ZShkYXRhKVxyXG5cclxuICAgIGRlZmVycmVkLnByb21pc2VcclxuXHJcbiAgQFxyXG4iLCJhbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKS5zZXJ2aWNlKCdKb2JNYW5hZ2VyQ29uZmlnU2VydmljZScsIGZ1bmN0aW9uKCRodHRwLCBmbGlua0NvbmZpZywgJHEpIHtcbiAgdmFyIGNvbmZpZztcbiAgY29uZmlnID0ge307XG4gIHRoaXMubG9hZENvbmZpZyA9IGZ
 1bmN0aW9uKCkge1xuICAgIHZhciBkZWZlcnJlZDtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJGh0dHAuZ2V0KFwiam9ibWFuYWdlci9jb25maWdcIikuc3VjY2VzcyhmdW5jdGlvbihkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykge1xuICAgICAgY29uZmlnID0gZGF0YTtcbiAgICAgIHJldHVybiBkZWZlcnJlZC5yZXNvbHZlKGRhdGEpO1xuICAgIH0pO1xuICAgIHJldHVybiBkZWZlcnJlZC5wcm9taXNlO1xuICB9O1xuICByZXR1cm4gdGhpcztcbn0pLnNlcnZpY2UoJ0pvYk1hbmFnZXJMb2dzU2VydmljZScsIGZ1bmN0aW9uKCRodHRwLCBmbGlua0NvbmZpZywgJHEpIHtcbiAgdmFyIGxvZ3M7XG4gIGxvZ3MgPSB7fTtcbiAgdGhpcy5sb2FkTG9ncyA9IGZ1bmN0aW9uKCkge1xuICAgIHZhciBkZWZlcnJlZDtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJGh0dHAuZ2V0KFwiam9ibWFuYWdlci9sb2dcIikuc3VjY2VzcyhmdW5jdGlvbihkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykge1xuICAgICAgbG9ncyA9IGRhdGE7XG4gICAgICByZXR1cm4gZGVmZXJyZWQucmVzb2x2ZShkYXRhKTtcbiAgICB9KTtcbiAgICByZXR1cm4gZGVmZXJyZWQucHJvbWlzZTtcbiAgfTtcbiAgcmV0dXJuIHRoaXM7XG59KS5zZXJ2aWNlKCdKb2JNYW5hZ2VyU3Rkb3V0U2VydmljZScsIGZ1bmN0aW9uKCRodHRwLCBmbGlua0NvbmZpZywgJHEpIHtcbiAgdmFyIHN0ZG91dDtcbiAgc3Rkb3V0
 ID0ge307XG4gIHRoaXMubG9hZFN0ZG91dCA9IGZ1bmN0aW9uKCkge1xuICAgIHZhciBkZWZlcnJlZDtcbiAgICBkZWZlcnJlZCA9ICRxLmRlZmVyKCk7XG4gICAgJGh0dHAuZ2V0KFwiam9ibWFuYWdlci9zdGRvdXRcIikuc3VjY2VzcyhmdW5jdGlvbihkYXRhLCBzdGF0dXMsIGhlYWRlcnMsIGNvbmZpZykge1xuICAgICAgc3Rkb3V0ID0gZGF0YTtcbiAgICAgIHJldHVybiBkZWZlcnJlZC5yZXNvbHZlKGRhdGEpO1xuICAgIH0pO1xuICAgIHJldHVybiBkZWZlcnJlZC5wcm9taXNlO1xuICB9O1xuICByZXR1cm4gdGhpcztcbn0pO1xuIiwiI1xyXG4jIExpY2Vuc2VkIHRvIHRoZSBBcGFjaGUgU29mdHdhcmUgRm91bmRhdGlvbiAoQVNGKSB1bmRlciBvbmVcclxuIyBvciBtb3JlIGNvbnRyaWJ1dG9yIGxpY2Vuc2UgYWdyZWVtZW50cy4gIFNlZSB0aGUgTk9USUNFIGZpbGVcclxuIyBkaXN0cmlidXRlZCB3aXRoIHRoaXMgd29yayBmb3IgYWRkaXRpb25hbCBpbmZvcm1hdGlvblxyXG4jIHJlZ2FyZGluZyBjb3B5cmlnaHQgb3duZXJzaGlwLiAgVGhlIEFTRiBsaWNlbnNlcyB0aGlzIGZpbGVcclxuIyB0byB5b3UgdW5kZXIgdGhlIEFwYWNoZSBMaWNlbnNlLCBWZXJzaW9uIDIuMCAodGhlXHJcbiMgXCJMaWNlbnNlXCIpOyB5b3UgbWF5IG5vdCB1c2UgdGhpcyBmaWxlIGV4Y2VwdCBpbiBjb21wbGlhbmNlXHJcbiMgd2l0aCB0aGUgTGljZW5zZS4gIFlvdSBtYXkgb2J0YWluIGEgY29weSBvZiB0aGUgTGljZW5zZSBhdFxyX
 G4jXHJcbiMgICAgIGh0dHA6Ly93d3cuYXBhY2hlLm9yZy9saWNlbnNlcy9MSUNFTlNFLTIuMFxyXG4jXHJcbiMgVW5sZXNzIHJlcXVpcmVkIGJ5IGFwcGxpY2FibGUgbGF3IG9yIGFncmVlZCB0byBpbiB3cml0aW5nLCBzb2Z0d2FyZVxyXG4jIGRpc3RyaWJ1dGVkIHVuZGVyIHRoZSBMaWNlbnNlIGlzIGRpc3RyaWJ1dGVkIG9uIGFuIFwiQVMgSVNcIiBCQVNJUyxcclxuIyBXSVRIT1VUIFdBUlJBTlRJRVMgT1IgQ09ORElUSU9OUyBPRiBBTlkgS0lORCwgZWl0aGVyIGV4cHJlc3Mgb3IgaW1wbGllZC5cclxuIyBTZWUgdGhlIExpY2Vuc2UgZm9yIHRoZSBzcGVjaWZpYyBsYW5ndWFnZSBnb3Zlcm5pbmcgcGVybWlzc2lvbnMgYW5kXHJcbiMgbGltaXRhdGlvbnMgdW5kZXIgdGhlIExpY2Vuc2UuXHJcbiNcclxuXHJcbmFuZ3VsYXIubW9kdWxlKCdmbGlua0FwcCcpXHJcblxyXG4uY29udHJvbGxlciAnUnVubmluZ0pvYnNDb250cm9sbGVyJywgKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSAtPlxyXG4gICRzY29wZS5qb2JPYnNlcnZlciA9IC0+XHJcbiAgICAkc2NvcGUuam9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ3J1bm5pbmcnKVxyXG5cclxuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcclxuICAkc2NvcGUuJG9uICckZGVzdHJveScsIC0+XHJcbiAgICBKb2JzU2VydmljZS51blJlZ2lzdGVyT2JzZXJ2ZXIoJHNjb3BlLmpvYk9ic2Vydm
 VyKVxyXG5cclxuICAkc2NvcGUuam9iT2JzZXJ2ZXIoKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbnRyb2xsZXIgJ0NvbXBsZXRlZEpvYnNDb250cm9sbGVyJywgKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSAtPlxyXG4gICRzY29wZS5qb2JPYnNlcnZlciA9IC0+XHJcbiAgICAkc2NvcGUuam9icyA9IEpvYnNTZXJ2aWNlLmdldEpvYnMoJ2ZpbmlzaGVkJylcclxuXHJcbiAgSm9ic1NlcnZpY2UucmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpXHJcbiAgJHNjb3BlLiRvbiAnJGRlc3Ryb3knLCAtPlxyXG4gICAgSm9ic1NlcnZpY2UudW5SZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcilcclxuXHJcbiAgJHNjb3BlLmpvYk9ic2VydmVyKClcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5jb250cm9sbGVyICdTaW5nbGVKb2JDb250cm9sbGVyJywgKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlLCAkcm9vdFNjb3BlLCBmbGlua0NvbmZpZywgJGludGVydmFsKSAtPlxyXG4gIGNvbnNvbGUubG9nICdTaW5nbGVKb2JDb250cm9sbGVyJ1xyXG5cclxuICAkc2NvcGUuam9iaWQgPSAkc3RhdGVQYXJhbXMuam9iaWRcclxuICAkc2NvcGUuam9iID0gbnVsbFxyXG4gICRzY29wZS5wbGFuID0gbnVsbFxyXG4gICRzY29wZS5
 2ZXJ0aWNlcyA9IG51bGxcclxuXHJcbiAgSm9ic1NlcnZpY2UubG9hZEpvYigkc3RhdGVQYXJhbXMuam9iaWQpLnRoZW4gKGRhdGEpIC0+XHJcbiAgICAkc2NvcGUuam9iID0gZGF0YVxyXG4gICAgJHNjb3BlLnBsYW4gPSBkYXRhLnBsYW5cclxuICAgICRzY29wZS52ZXJ0aWNlcyA9IGRhdGEudmVydGljZXNcclxuXHJcbiAgcmVmcmVzaGVyID0gJGludGVydmFsIC0+XHJcbiAgICBKb2JzU2VydmljZS5sb2FkSm9iKCRzdGF0ZVBhcmFtcy5qb2JpZCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgJHNjb3BlLmpvYiA9IGRhdGFcclxuXHJcbiAgICAgICRzY29wZS4kYnJvYWRjYXN0ICdyZWxvYWQnXHJcblxyXG4gICwgZmxpbmtDb25maWdbXCJyZWZyZXNoLWludGVydmFsXCJdXHJcblxyXG4gICRzY29wZS4kb24gJyRkZXN0cm95JywgLT5cclxuICAgICRzY29wZS5qb2IgPSBudWxsXHJcbiAgICAkc2NvcGUucGxhbiA9IG51bGxcclxuICAgICRzY29wZS52ZXJ0aWNlcyA9IG51bGxcclxuXHJcbiAgICAkaW50ZXJ2YWwuY2FuY2VsKHJlZnJlc2hlcilcclxuXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4uY29udHJvbGxlciAnSm9iUGxhbkNvbnRyb2xsZXInLCAoJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIC0+XHJcbiAgY29uc29sZS5sb2cgJ0pvYlBsYW5Db250cm9sbGVyJ1xyXG5cclxuICAkc2NvcGUubm9kZWlkID0gbnVsbFxyXG4g
 ICRzY29wZS5ub2RlVW5mb2xkZWQgPSBmYWxzZVxyXG4gICRzY29wZS5zdGF0ZUxpc3QgPSBKb2JzU2VydmljZS5zdGF0ZUxpc3QoKVxyXG5cclxuICAkc2NvcGUuY2hhbmdlTm9kZSA9IChub2RlaWQpIC0+XHJcbiAgICBpZiBub2RlaWQgIT0gJHNjb3BlLm5vZGVpZFxyXG4gICAgICAkc2NvcGUubm9kZWlkID0gbm9kZWlkXHJcbiAgICAgICRzY29wZS52ZXJ0ZXggPSBudWxsXHJcbiAgICAgICRzY29wZS5zdWJ0YXNrcyA9IG51bGxcclxuICAgICAgJHNjb3BlLmFjY3VtdWxhdG9ycyA9IG51bGxcclxuXHJcbiAgICAgICRzY29wZS4kYnJvYWRjYXN0ICdyZWxvYWQnXHJcblxyXG4gICAgZWxzZVxyXG4gICAgICAkc2NvcGUubm9kZWlkID0gbnVsbFxyXG4gICAgICAkc2NvcGUubm9kZVVuZm9sZGVkID0gZmFsc2VcclxuICAgICAgJHNjb3BlLnZlcnRleCA9IG51bGxcclxuICAgICAgJHNjb3BlLnN1YnRhc2tzID0gbnVsbFxyXG4gICAgICAkc2NvcGUuYWNjdW11bGF0b3JzID0gbnVsbFxyXG5cclxuICAkc2NvcGUuZGVhY3RpdmF0ZU5vZGUgPSAtPlxyXG4gICAgJHNjb3BlLm5vZGVpZCA9IG51bGxcclxuICAgICRzY29wZS5ub2RlVW5mb2xkZWQgPSBmYWxzZVxyXG4gICAgJHNjb3BlLnZlcnRleCA9IG51bGxcclxuICAgICRzY29wZS5zdWJ0YXNrcyA9IG51bGxcclxuICAgICRzY29wZS5hY2N1bXVsYXRvcnMgPSBudWxsXHJcblxyXG4gICRzY29wZS50b2dnbGVGb2xkID0gLT5cclxuICAgICRzY29wZS5ub2RlVW5mb
 2xkZWQgPSAhJHNjb3BlLm5vZGVVbmZvbGRlZFxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLVxyXG5cclxuLmNvbnRyb2xsZXIgJ0pvYlBsYW5PdmVydmlld0NvbnRyb2xsZXInLCAoJHNjb3BlLCBKb2JzU2VydmljZSkgLT5cclxuICBjb25zb2xlLmxvZyAnSm9iUGxhbk92ZXJ2aWV3Q29udHJvbGxlcidcclxuXHJcbiAgaWYgJHNjb3BlLm5vZGVpZCBhbmQgKCEkc2NvcGUudmVydGV4IG9yICEkc2NvcGUudmVydGV4LnN0KVxyXG4gICAgSm9ic1NlcnZpY2UuZ2V0U3VidGFza3MoJHNjb3BlLm5vZGVpZCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgJHNjb3BlLnN1YnRhc2tzID0gZGF0YVxyXG5cclxuICAkc2NvcGUuJG9uICdyZWxvYWQnLCAoZXZlbnQpIC0+XHJcbiAgICBjb25zb2xlLmxvZyAnSm9iUGxhbk92ZXJ2aWV3Q29udHJvbGxlcidcclxuICAgIGlmICRzY29wZS5ub2RlaWRcclxuICAgICAgSm9ic1NlcnZpY2UuZ2V0U3VidGFza3MoJHNjb3BlLm5vZGVpZCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgICAkc2NvcGUuc3VidGFza3MgPSBkYXRhXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4uY29udHJvbGxlciAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInLCAoJHNjb3BlLCBKb2JzU2VydmljZSkgLT5cclxuICBjb25zb2xlLmxvZyAnSm9iUGxhbkFjY3VtdWxhdG9yc0NvbnRyb2xsZXInXHJcblxyXG4gIG
 lmICRzY29wZS5ub2RlaWQgYW5kICghJHNjb3BlLnZlcnRleCBvciAhJHNjb3BlLnZlcnRleC5hY2N1bXVsYXRvcnMpXHJcbiAgICBKb2JzU2VydmljZS5nZXRBY2N1bXVsYXRvcnMoJHNjb3BlLm5vZGVpZCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgJHNjb3BlLmFjY3VtdWxhdG9ycyA9IGRhdGEubWFpblxyXG4gICAgICAkc2NvcGUuc3VidGFza0FjY3VtdWxhdG9ycyA9IGRhdGEuc3VidGFza3NcclxuXHJcbiAgJHNjb3BlLiRvbiAncmVsb2FkJywgKGV2ZW50KSAtPlxyXG4gICAgY29uc29sZS5sb2cgJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJ1xyXG4gICAgaWYgJHNjb3BlLm5vZGVpZFxyXG4gICAgICBKb2JzU2VydmljZS5nZXRBY2N1bXVsYXRvcnMoJHNjb3BlLm5vZGVpZCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgICAkc2NvcGUuYWNjdW11bGF0b3JzID0gZGF0YS5tYWluXHJcbiAgICAgICAgJHNjb3BlLnN1YnRhc2tBY2N1bXVsYXRvcnMgPSBkYXRhLnN1YnRhc2tzXHJcblxyXG4jIC0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4uY29udHJvbGxlciAnSm9iVGltZWxpbmVWZXJ0ZXhDb250cm9sbGVyJywgKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlKSAtPlxyXG4gIGNvbnNvbGUubG9nICdKb2JUaW1lbGluZVZlcnRleENvbnRyb2xsZXInXHJcblxyXG4gIEpvYnNTZXJ2aWNlLmdldFZlcnRleCgkc3RhdGVQYXJhbXMudmVydGV
 4SWQpLnRoZW4gKGRhdGEpIC0+XHJcbiAgICAkc2NvcGUudmVydGV4ID0gZGF0YVxyXG5cclxuICAkc2NvcGUuJG9uICdyZWxvYWQnLCAoZXZlbnQpIC0+XHJcbiAgICBjb25zb2xlLmxvZyAnSm9iVGltZWxpbmVWZXJ0ZXhDb250cm9sbGVyJ1xyXG4gICAgSm9ic1NlcnZpY2UuZ2V0VmVydGV4KCRzdGF0ZVBhcmFtcy52ZXJ0ZXhJZCkudGhlbiAoZGF0YSkgLT5cclxuICAgICAgJHNjb3BlLnZlcnRleCA9IGRhdGFcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5jb250cm9sbGVyICdKb2JFeGNlcHRpb25zQ29udHJvbGxlcicsICgkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkgLT5cclxuICBKb2JzU2VydmljZS5sb2FkRXhjZXB0aW9ucygpLnRoZW4gKGRhdGEpIC0+XHJcbiAgICAkc2NvcGUuZXhjZXB0aW9ucyA9IGRhdGFcclxuXHJcbiMgLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS1cclxuXHJcbi5jb250cm9sbGVyICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcicsICgkc2NvcGUsIEpvYnNTZXJ2aWNlKSAtPlxyXG4gIGNvbnNvbGUubG9nICdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcidcclxuXHJcbiAgJHNjb3BlLmNoYW5nZU5vZGUgPSAobm9kZWlkKSAtPlxyXG4gICAgaWYgbm9kZWlkICE9ICRzY29wZS5ub2RlaWRcclxuICAgICAgJHNjb3BlLm5vZGVpZCA9IG5vZGVpZFxyXG5cclxuICAgICAgSm9ic1NlcnZp
 Y2UuZ2V0Tm9kZShub2RlaWQpLnRoZW4gKGRhdGEpIC0+XHJcbiAgICAgICAgJHNjb3BlLm5vZGUgPSBkYXRhXHJcblxyXG4gICAgZWxzZVxyXG4gICAgICAkc2NvcGUubm9kZWlkID0gbnVsbFxyXG4gICAgICAkc2NvcGUubm9kZSA9IG51bGxcclxuIiwiYW5ndWxhci5tb2R1bGUoJ2ZsaW5rQXBwJykuY29udHJvbGxlcignUnVubmluZ0pvYnNDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIHtcbiAgJHNjb3BlLmpvYk9ic2VydmVyID0gZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuICRzY29wZS5qb2JzID0gSm9ic1NlcnZpY2UuZ2V0Sm9icygncnVubmluZycpO1xuICB9O1xuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5qb2JPYnNlcnZlcigpO1xufSkuY29udHJvbGxlcignQ29tcGxldGVkSm9ic0NvbnRyb2xsZXInLCBmdW5jdGlvbigkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkge1xuICAkc2NvcGUuam9iT2JzZXJ2ZXIgPSBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gJHNjb3BlLmpvYnMgPSBKb2JzU2VydmljZS5nZXRKb2JzKCdmaW5pc2hlZCcpO
 1xuICB9O1xuICBKb2JzU2VydmljZS5yZWdpc3Rlck9ic2VydmVyKCRzY29wZS5qb2JPYnNlcnZlcik7XG4gICRzY29wZS4kb24oJyRkZXN0cm95JywgZnVuY3Rpb24oKSB7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLnVuUmVnaXN0ZXJPYnNlcnZlcigkc2NvcGUuam9iT2JzZXJ2ZXIpO1xuICB9KTtcbiAgcmV0dXJuICRzY29wZS5qb2JPYnNlcnZlcigpO1xufSkuY29udHJvbGxlcignU2luZ2xlSm9iQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgJHN0YXRlLCAkc3RhdGVQYXJhbXMsIEpvYnNTZXJ2aWNlLCAkcm9vdFNjb3BlLCBmbGlua0NvbmZpZywgJGludGVydmFsKSB7XG4gIHZhciByZWZyZXNoZXI7XG4gIGNvbnNvbGUubG9nKCdTaW5nbGVKb2JDb250cm9sbGVyJyk7XG4gICRzY29wZS5qb2JpZCA9ICRzdGF0ZVBhcmFtcy5qb2JpZDtcbiAgJHNjb3BlLmpvYiA9IG51bGw7XG4gICRzY29wZS5wbGFuID0gbnVsbDtcbiAgJHNjb3BlLnZlcnRpY2VzID0gbnVsbDtcbiAgSm9ic1NlcnZpY2UubG9hZEpvYigkc3RhdGVQYXJhbXMuam9iaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICRzY29wZS5qb2IgPSBkYXRhO1xuICAgICRzY29wZS5wbGFuID0gZGF0YS5wbGFuO1xuICAgIHJldHVybiAkc2NvcGUudmVydGljZXMgPSBkYXRhLnZlcnRpY2VzO1xuICB9KTtcbiAgcmVmcmVzaGVyID0gJGludGVydmFsKGZ1bmN0aW9uKCkge1xuICAgIHJldHVybiBKb2JzU2VydmljZS5sb2FkSm9iKCRzdGF0ZVBhcm
 Ftcy5qb2JpZCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgICAkc2NvcGUuam9iID0gZGF0YTtcbiAgICAgIHJldHVybiAkc2NvcGUuJGJyb2FkY2FzdCgncmVsb2FkJyk7XG4gICAgfSk7XG4gIH0sIGZsaW5rQ29uZmlnW1wicmVmcmVzaC1pbnRlcnZhbFwiXSk7XG4gIHJldHVybiAkc2NvcGUuJG9uKCckZGVzdHJveScsIGZ1bmN0aW9uKCkge1xuICAgICRzY29wZS5qb2IgPSBudWxsO1xuICAgICRzY29wZS5wbGFuID0gbnVsbDtcbiAgICAkc2NvcGUudmVydGljZXMgPSBudWxsO1xuICAgIHJldHVybiAkaW50ZXJ2YWwuY2FuY2VsKHJlZnJlc2hlcik7XG4gIH0pO1xufSkuY29udHJvbGxlcignSm9iUGxhbkNvbnRyb2xsZXInLCBmdW5jdGlvbigkc2NvcGUsICRzdGF0ZSwgJHN0YXRlUGFyYW1zLCBKb2JzU2VydmljZSkge1xuICBjb25zb2xlLmxvZygnSm9iUGxhbkNvbnRyb2xsZXInKTtcbiAgJHNjb3BlLm5vZGVpZCA9IG51bGw7XG4gICRzY29wZS5ub2RlVW5mb2xkZWQgPSBmYWxzZTtcbiAgJHNjb3BlLnN0YXRlTGlzdCA9IEpvYnNTZXJ2aWNlLnN0YXRlTGlzdCgpO1xuICAkc2NvcGUuY2hhbmdlTm9kZSA9IGZ1bmN0aW9uKG5vZGVpZCkge1xuICAgIGlmIChub2RlaWQgIT09ICRzY29wZS5ub2RlaWQpIHtcbiAgICAgICRzY29wZS5ub2RlaWQgPSBub2RlaWQ7XG4gICAgICAkc2NvcGUudmVydGV4ID0gbnVsbDtcbiAgICAgICRzY29wZS5zdWJ0YXNrcyA9IG51bGw7XG4gICAgICAkc2NvcGUuYWNjdW11bGF
 0b3JzID0gbnVsbDtcbiAgICAgIHJldHVybiAkc2NvcGUuJGJyb2FkY2FzdCgncmVsb2FkJyk7XG4gICAgfSBlbHNlIHtcbiAgICAgICRzY29wZS5ub2RlaWQgPSBudWxsO1xuICAgICAgJHNjb3BlLm5vZGVVbmZvbGRlZCA9IGZhbHNlO1xuICAgICAgJHNjb3BlLnZlcnRleCA9IG51bGw7XG4gICAgICAkc2NvcGUuc3VidGFza3MgPSBudWxsO1xuICAgICAgcmV0dXJuICRzY29wZS5hY2N1bXVsYXRvcnMgPSBudWxsO1xuICAgIH1cbiAgfTtcbiAgJHNjb3BlLmRlYWN0aXZhdGVOb2RlID0gZnVuY3Rpb24oKSB7XG4gICAgJHNjb3BlLm5vZGVpZCA9IG51bGw7XG4gICAgJHNjb3BlLm5vZGVVbmZvbGRlZCA9IGZhbHNlO1xuICAgICRzY29wZS52ZXJ0ZXggPSBudWxsO1xuICAgICRzY29wZS5zdWJ0YXNrcyA9IG51bGw7XG4gICAgcmV0dXJuICRzY29wZS5hY2N1bXVsYXRvcnMgPSBudWxsO1xuICB9O1xuICByZXR1cm4gJHNjb3BlLnRvZ2dsZUZvbGQgPSBmdW5jdGlvbigpIHtcbiAgICByZXR1cm4gJHNjb3BlLm5vZGVVbmZvbGRlZCA9ICEkc2NvcGUubm9kZVVuZm9sZGVkO1xuICB9O1xufSkuY29udHJvbGxlcignSm9iUGxhbk92ZXJ2aWV3Q29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgSm9ic1NlcnZpY2UpIHtcbiAgY29uc29sZS5sb2coJ0pvYlBsYW5PdmVydmlld0NvbnRyb2xsZXInKTtcbiAgaWYgKCRzY29wZS5ub2RlaWQgJiYgKCEkc2NvcGUudmVydGV4IHx8ICEkc2NvcGUudmVydGV4LnN0KSkge1xuICAg
 IEpvYnNTZXJ2aWNlLmdldFN1YnRhc2tzKCRzY29wZS5ub2RlaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICAgcmV0dXJuICRzY29wZS5zdWJ0YXNrcyA9IGRhdGE7XG4gICAgfSk7XG4gIH1cbiAgcmV0dXJuICRzY29wZS4kb24oJ3JlbG9hZCcsIGZ1bmN0aW9uKGV2ZW50KSB7XG4gICAgY29uc29sZS5sb2coJ0pvYlBsYW5PdmVydmlld0NvbnRyb2xsZXInKTtcbiAgICBpZiAoJHNjb3BlLm5vZGVpZCkge1xuICAgICAgcmV0dXJuIEpvYnNTZXJ2aWNlLmdldFN1YnRhc2tzKCRzY29wZS5ub2RlaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICAgICByZXR1cm4gJHNjb3BlLnN1YnRhc2tzID0gZGF0YTtcbiAgICAgIH0pO1xuICAgIH1cbiAgfSk7XG59KS5jb250cm9sbGVyKCdKb2JQbGFuQWNjdW11bGF0b3JzQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgSm9ic1NlcnZpY2UpIHtcbiAgY29uc29sZS5sb2coJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJyk7XG4gIGlmICgkc2NvcGUubm9kZWlkICYmICghJHNjb3BlLnZlcnRleCB8fCAhJHNjb3BlLnZlcnRleC5hY2N1bXVsYXRvcnMpKSB7XG4gICAgSm9ic1NlcnZpY2UuZ2V0QWNjdW11bGF0b3JzKCRzY29wZS5ub2RlaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICAgJHNjb3BlLmFjY3VtdWxhdG9ycyA9IGRhdGEubWFpbjtcbiAgICAgIHJldHVybiAkc2NvcGUuc3VidGFza0FjY3VtdWxhdG9ycyA9IGRhdGEuc3VidGFza3M7X
 G4gICAgfSk7XG4gIH1cbiAgcmV0dXJuICRzY29wZS4kb24oJ3JlbG9hZCcsIGZ1bmN0aW9uKGV2ZW50KSB7XG4gICAgY29uc29sZS5sb2coJ0pvYlBsYW5BY2N1bXVsYXRvcnNDb250cm9sbGVyJyk7XG4gICAgaWYgKCRzY29wZS5ub2RlaWQpIHtcbiAgICAgIHJldHVybiBKb2JzU2VydmljZS5nZXRBY2N1bXVsYXRvcnMoJHNjb3BlLm5vZGVpZCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgICAgICRzY29wZS5hY2N1bXVsYXRvcnMgPSBkYXRhLm1haW47XG4gICAgICAgIHJldHVybiAkc2NvcGUuc3VidGFza0FjY3VtdWxhdG9ycyA9IGRhdGEuc3VidGFza3M7XG4gICAgICB9KTtcbiAgICB9XG4gIH0pO1xufSkuY29udHJvbGxlcignSm9iVGltZWxpbmVWZXJ0ZXhDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIHtcbiAgY29uc29sZS5sb2coJ0pvYlRpbWVsaW5lVmVydGV4Q29udHJvbGxlcicpO1xuICBKb2JzU2VydmljZS5nZXRWZXJ0ZXgoJHN0YXRlUGFyYW1zLnZlcnRleElkKS50aGVuKGZ1bmN0aW9uKGRhdGEpIHtcbiAgICByZXR1cm4gJHNjb3BlLnZlcnRleCA9IGRhdGE7XG4gIH0pO1xuICByZXR1cm4gJHNjb3BlLiRvbigncmVsb2FkJywgZnVuY3Rpb24oZXZlbnQpIHtcbiAgICBjb25zb2xlLmxvZygnSm9iVGltZWxpbmVWZXJ0ZXhDb250cm9sbGVyJyk7XG4gICAgcmV0dXJuIEpvYnNTZXJ2aWNlLmdldFZlcnRleCgkc3RhdGVQYXJhbX
 MudmVydGV4SWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICAgcmV0dXJuICRzY29wZS52ZXJ0ZXggPSBkYXRhO1xuICAgIH0pO1xuICB9KTtcbn0pLmNvbnRyb2xsZXIoJ0pvYkV4Y2VwdGlvbnNDb250cm9sbGVyJywgZnVuY3Rpb24oJHNjb3BlLCAkc3RhdGUsICRzdGF0ZVBhcmFtcywgSm9ic1NlcnZpY2UpIHtcbiAgcmV0dXJuIEpvYnNTZXJ2aWNlLmxvYWRFeGNlcHRpb25zKCkudGhlbihmdW5jdGlvbihkYXRhKSB7XG4gICAgcmV0dXJuICRzY29wZS5leGNlcHRpb25zID0gZGF0YTtcbiAgfSk7XG59KS5jb250cm9sbGVyKCdKb2JQcm9wZXJ0aWVzQ29udHJvbGxlcicsIGZ1bmN0aW9uKCRzY29wZSwgSm9ic1NlcnZpY2UpIHtcbiAgY29uc29sZS5sb2coJ0pvYlByb3BlcnRpZXNDb250cm9sbGVyJyk7XG4gIHJldHVybiAkc2NvcGUuY2hhbmdlTm9kZSA9IGZ1bmN0aW9uKG5vZGVpZCkge1xuICAgIGlmIChub2RlaWQgIT09ICRzY29wZS5ub2RlaWQpIHtcbiAgICAgICRzY29wZS5ub2RlaWQgPSBub2RlaWQ7XG4gICAgICByZXR1cm4gSm9ic1NlcnZpY2UuZ2V0Tm9kZShub2RlaWQpLnRoZW4oZnVuY3Rpb24oZGF0YSkge1xuICAgICAgICByZXR1cm4gJHNjb3BlLm5vZGUgPSBkYXRhO1xuICAgICAgfSk7XG4gICAgfSBlbHNlIHtcbiAgICAgICRzY29wZS5ub2RlaWQgPSBudWxsO1xuICAgICAgcmV0dXJuICRzY29wZS5ub2RlID0gbnVsbDtcbiAgICB9XG4gIH07XG59KTtcbiIsIiNcclxuIyBMaWNlbnNlZCB
 0byB0aGUgQXBhY2hlIFNvZnR3YXJlIEZvdW5kYXRpb24gKEFTRikgdW5kZXIgb25lXHJcbiMgb3IgbW9yZSBjb250cmlidXRvciBsaWNlbnNlIGFncmVlbWVudHMuICBTZWUgdGhlIE5PVElDRSBmaWxlXHJcbiMgZGlzdHJpYnV0ZWQgd2l0aCB0aGlzIHdvcmsgZm9yIGFkZGl0aW9uYWwgaW5mb3JtYXRpb25cclxuIyByZWdhcmRpbmcgY29weXJpZ2h0IG93bmVyc2hpcC4gIFRoZSBBU0YgbGljZW5zZXMgdGhpcyBmaWxlXHJcbiMgdG8geW91IHVuZGVyIHRoZSBBcGFjaGUgTGljZW5zZSwgVmVyc2lvbiAyLjAgKHRoZVxyXG4jIFwiTGljZW5zZVwiKTsgeW91IG1heSBub3QgdXNlIHRoaXMgZmlsZSBleGNlcHQgaW4gY29tcGxpYW5jZVxyXG4jIHdpdGggdGhlIExpY2Vuc2UuICBZb3UgbWF5IG9idGFpbiBhIGNvcHkgb2YgdGhlIExpY2Vuc2UgYXRcclxuI1xyXG4jICAgICBodHRwOi8vd3d3LmFwYWNoZS5vcmcvbGljZW5zZXMvTElDRU5TRS0yLjBcclxuI1xyXG4jIFVubGVzcyByZXF1aXJlZCBieSBhcHBsaWNhYmxlIGxhdyBvciBhZ3JlZWQgdG8gaW4gd3JpdGluZywgc29mdHdhcmVcclxuIyBkaXN0cmlidXRlZCB1bmRlciB0aGUgTGljZW5zZSBpcyBkaXN0cmlidXRlZCBvbiBhbiBcIkFTIElTXCIgQkFTSVMsXHJcbiMgV0lUSE9VVCBXQVJSQU5USUVTIE9SIENPTkRJVElPTlMgT0YgQU5ZIEtJTkQsIGVpdGhlciBleHByZXNzIG9yIGltcGxpZWQuXHJcbiMgU2VlIHRoZSBMaWNlbnNlIGZvciB0aGUgc3BlY2lmaWMg
 bGFuZ3VhZ2UgZ292ZXJuaW5nIHBlcm1pc3Npb25zIGFuZFxyXG4jIGxpbWl0YXRpb25zIHVuZGVyIHRoZSBMaWNlbnNlLlxyXG4jXHJcblxyXG5hbmd1bGFyLm1vZHVsZSgnZmxpbmtBcHAnKVxyXG5cclxuIyAtLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tLS0tXHJcblxyXG4uZGlyZWN0aXZlICd2ZXJ0ZXgnLCAoJHN0YXRlKSAtPlxyXG4gIHRlbXBsYXRlOiBcIjxzdmcgY2xhc3M9J3RpbWVsaW5lIHNlY29uZGFyeScgd2lkdGg9JzAnIGhlaWdodD0nMCc+PC9zdmc+XCJcclxuXHJcbiAgc2NvcGU6XHJcbiAgICBkYXRhOiBcIj1cIlxyXG5cclxuICBsaW5rOiAoc2NvcGUsIGVsZW0sIGF0dHJzKSAtPlxyXG4gICAgc3ZnRWwgPSBlbGVtLmNoaWxkcmVuKClbMF1cclxuXHJcbiAgICBjb250YWluZXJXID0gZWxlbS53aWR0aCgpXHJcbiAgICBhbmd1bGFyLmVsZW1lbnQoc3ZnRWwpLmF0dHIoJ3dpZHRoJywgY29udGFpbmVyVylcclxuXHJcbiAgICBhbmFseXplVGltZSA9IChkYXRhKSAtPlxyXG4gICAgICBkMy5zZWxlY3Qoc3ZnRWwpLnNlbGVjdEFsbChcIipcIikucmVtb3ZlKClcclxuXHJcbiAgICAgIHRlc3REYXRhID0gW11cclxuXHJcbiAgICAgIGFuZ3VsYXIuZm9yRWFjaCBkYXRhLnN1YnRhc2tzLCAoc3VidGFzaywgaSkgLT5cclxuICAgICAgICB0aW1lcyA9IFtcclxuICAgICAgICAgIHtcclxuICAgICAgICAgICAgbGFiZWw6IFwiU2NoZWR1bGVkXCJcclxuICAgICAgICAgICAgY
 29sb3I6IFwiIzY2NlwiXHJcbiAgICAgICAgICAgIGJvcmRlckNvbG9yOiBcIiM1NTVcIlxyXG4gICAgICAgICAgICBzdGFydGluZ190aW1lOiBzdWJ0YXNrLnRpbWVzdGFtcHNbXCJTQ0hFRFVMRURcIl1cclxuICAgICAgICAgICAgZW5kaW5nX3RpbWU6IHN1YnRhc2sudGltZXN0YW1wc1tcIkRFUExPWUlOR1wiXVxyXG4gICAgICAgICAgICB0eXBlOiAncmVndWxhcidcclxuICAgICAgICAgIH1cclxuICAgICAgICAgIHtcclxuICAgICAgICAgICAgbGFiZWw6IFwiRGVwbG95aW5nXCJcclxuICAgICAgICAgICAgY29sb3I6IFwiI2FhYVwiXHJcbiAgICAgICAgICAgIGJvcmRlckNvbG9yOiBcIiM1NTVcIlxyXG4gICAgICAgICAgICBzdGFydGluZ190aW1lOiBzdWJ0YXNrLnRpbWVzdGFtcHNbXCJERVBMT1lJTkdcIl1cclxuICAgICAgICAgICAgZW5kaW5nX3RpbWU6IHN1YnRhc2sudGltZXN0YW1wc1tcIlJVTk5JTkdcIl1cclxuICAgICAgICAgICAgdHlwZTogJ3JlZ3VsYXInXHJcbiAgICAgICAgICB9XHJcbiAgICAgICAgXVxyXG5cclxuICAgICAgICBpZiBzdWJ0YXNrLnRpbWVzdGFtcHNbXCJGSU5JU0hFRFwiXSA+IDBcclxuICAgICAgICAgIHRpbWVzLnB1c2gge1xyXG4gICAgICAgICAgICBsYWJlbDogXCJSdW5uaW5nXCJcclxuICAgICAgICAgICAgY29sb3I6IFwiI2RkZFwiXHJcbiAgICAgICAgICAgIGJvcmRlckNvbG9yOiBcIiM1NTVcIlxyXG4gICAgICAgICAgICBzdGFydGluZ190aW1lOiBzdWJ0YXNrLn
 RpbWVzdGFtcHNbXCJSVU5OSU5HXCJdXHJcbiAgICAgICAgICAgIGVuZGluZ190aW1lOiBzdWJ0YXNrLnRpbWVzdGFtcHNbXCJGSU5JU0hFRFwiXVxyXG4gICAgICAgICAgICB0eXBlOiAncmVndWxhcidcclxuICAgICAgICAgIH

<TRUNCATED>

[07/13] flink git commit: [FLINK-2559] Clean up JavaDocs

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
index 6f00cdb..e14e06d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java
@@ -33,7 +33,7 @@ import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 
 /**
- * Intermediate step of an Outer Join transformation. <br/>
+ * Intermediate step of an Outer Join transformation. <br>
  * To continue the Join transformation, select the join key of the first input {@link DataSet} by calling
  * {@link JoinOperatorSetsBase#where(int...)} or
  * {@link JoinOperatorSetsBase#where(KeySelector)}.
@@ -69,9 +69,9 @@ public class JoinOperatorSetsBase<I1, I2> {
 	}
 
 	/**
-	 * Continues a Join transformation. <br/>
-	 * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.<br/>
-	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+	 * Continues a Join transformation. <br>
+	 * Defines the {@link Tuple} fields of the first join {@link DataSet} that should be used as join keys.<br>
+	 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br>
 	 *
 	 * @param fields The indexes of the other Tuple fields of the first join DataSets that should be used as keys.
 	 * @return An incomplete Join transformation.
@@ -87,7 +87,7 @@ public class JoinOperatorSetsBase<I1, I2> {
 	}
 
 	/**
-	 * Continues a Join transformation. <br/>
+	 * Continues a Join transformation. <br>
 	 * Defines the fields of the first join {@link DataSet} that should be used as grouping keys. Fields
 	 * are the names of member fields of the underlying type of the data set.
 	 *
@@ -105,9 +105,9 @@ public class JoinOperatorSetsBase<I1, I2> {
 	}
 
 	/**
-	 * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.</br>
+	 * Continues a Join transformation and defines a {@link KeySelector} function for the first join {@link DataSet}.<br>
 	 * The KeySelector function is called for each element of the first DataSet and extracts a single
-	 * key value on which the DataSet is joined. </br>
+	 * key value on which the DataSet is joined. <br>
 	 *
 	 * @param keySelector The KeySelector function which extracts the key values from the DataSet on which it is joined.
 	 * @return An incomplete Join transformation.
@@ -125,7 +125,7 @@ public class JoinOperatorSetsBase<I1, I2> {
 
 
 	/**
-	 * Intermediate step of a Join transformation. <br/>
+	 * Intermediate step of a Join transformation. <br>
 	 * To continue the Join transformation, select the join key of the second input {@link DataSet} by calling
 	 * {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(int...)} or
 	 * {@link org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.JoinOperatorSetsPredicateBase#equalTo(KeySelector)}.
@@ -149,8 +149,8 @@ public class JoinOperatorSetsBase<I1, I2> {
 
 		/**
 		 * Continues a Join transformation and defines the {@link Tuple} fields of the second join
-		 * {@link DataSet} that should be used as join keys.<br/>
-		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br/>
+		 * {@link DataSet} that should be used as join keys.<br>
+		 * <b>Note: Fields can only be selected as join keys on Tuple DataSets.</b><br>
 		 *
 		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
 		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
@@ -164,7 +164,7 @@ public class JoinOperatorSetsBase<I1, I2> {
 
 		/**
 		 * Continues a Join transformation and defines the fields of the second join
-		 * {@link DataSet} that should be used as join keys.<br/>
+		 * {@link DataSet} that should be used as join keys.<br>
 		 *
 		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
 		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}
@@ -177,9 +177,9 @@ public class JoinOperatorSetsBase<I1, I2> {
 		}
 
 		/**
-		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.</br>
+		 * Continues a Join transformation and defines a {@link KeySelector} function for the second join {@link DataSet}.<br>
 		 * The KeySelector function is called for each element of the second DataSet and extracts a single
-		 * key value on which the DataSet is joined. </br>
+		 * key value on which the DataSet is joined. <br>
 		 *
 		 * The resulting {@link JoinFunctionAssigner} needs to be finished by providing a
 		 * {@link JoinFunction} by calling {@link JoinFunctionAssigner#with(JoinFunction)}

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/Tuple3WrappingCollector.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/Tuple3WrappingCollector.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/Tuple3WrappingCollector.java
index 66592f5..29d68f1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/Tuple3WrappingCollector.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/Tuple3WrappingCollector.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.util.Collector;
 
 /**
- * Needed to wrap tuples to Tuple3<groupKey, sortKey, value> for combine method of group reduce with key selector sorting
+ * Needed to wrap tuples to {@code Tuple3<groupKey, sortKey, value>} for combine method of group reduce with key selector sorting
  */
 public class Tuple3WrappingCollector<IN, K1, K2> implements Collector<IN>, java.io.Serializable {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java
index 43f667f..8f0b4fa 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleWrappingCollector.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.util.Collector;
 
 /**
- * Needed to wrap tuples to Tuple2<key, value> pairs for combine method of group reduce with key selector function
+ * Needed to wrap tuples to {@code Tuple2<key, value>} pairs for combine method of group reduce with key selector function
  */
 public class TupleWrappingCollector<IN, K> implements Collector<IN>, java.io.Serializable {
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/record/functions/FunctionAnnotation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/FunctionAnnotation.java b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/FunctionAnnotation.java
index d7a17b8..71d2a62 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/record/functions/FunctionAnnotation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/record/functions/FunctionAnnotation.java
@@ -44,8 +44,8 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper;
  * annotation for a map-type function that realizes a simple absolute function,
  * use it the following way:
  * 
- * <pre><blockquote>
- * \@ConstantFieldsExcept(fields={2})
+ * <pre>{@code
+ * {@literal @}ConstantFieldsExcept(fields={2})
  * public class MyMapper extends MapFunction
  * {
  *     public void map(Record record, Collector out)
@@ -56,7 +56,7 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper;
 		out.collect(record);
  *     }
  * }
- * </blockquote></pre>
+ * }</pre>
  * 
  * Be aware that some annotations should only be used for functions with as single input 
  * ({@link MapFunction}, {@link ReduceFunction}) and some only for stubs with two inputs 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormat.java
index b1f7ffc..cbf16b5 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormat.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormat.java
@@ -32,7 +32,7 @@ import org.apache.flink.core.io.GenericInputSplit;
  * The input format checks the exit code of the process to validate whether the process terminated correctly. A list of allowed exit codes can be provided.
  * The input format requires ({@link ExternalProcessInputSplit} objects that hold the command to execute.
  * 
- * <b>Attention! </b><br/>
+ * <b>Attention! </b><br>
  * You must take care to read from (and process) both output streams of the process, standard out (stdout) and standard error (stderr). 
  * Otherwise, the input format might get deadlocked! 
  * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
index 59c7eb0..5f74513 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
@@ -29,8 +29,8 @@ import java.util.List;
 /**
  * Special type information to generate a special AvroTypeInfo for Avro POJOs (implementing SpecificRecordBase, the typed Avro POJOs)
  *
- * Proceeding: It uses a regular pojo type analysis and replaces all GenericType<CharSequence>
- *     with a GenericType<avro.Utf8>.
+ * Proceeding: It uses a regular pojo type analysis and replaces all {@code GenericType<CharSequence>}
+ *     with a {@code GenericType<avro.Utf8>}.
  * All other types used by Avro are standard Java types.
  * Only strings are represented as CharSequence fields and represented as Utf8 classes at runtime.
  * CharSequence is not comparable. To make them nicely usable with field expressions, we replace them here

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
index 9e0da25..76f8eb4 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
@@ -107,8 +107,8 @@ public class Serializers {
 	}
 
 	/**
-	 * Register these serializers for using Avro's {@see GenericData.Record} and classes
-	 * implementing {@see org.apache.avro.specific.SpecificRecordBase}
+	 * Register these serializers for using Avro's {@link GenericData.Record} and classes
+	 * implementing {@link org.apache.avro.specific.SpecificRecordBase}
 	 */
 	public static void registerGenericAvro(ExecutionConfig reg) {
 		// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java
index 8f9fa3f..b91dc82 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java
@@ -206,7 +206,7 @@ public final class DataSetUtils {
 	 * <p>
 	 * <strong>NOTE:</strong> Sample with fixed size is not as efficient as sample with fraction, use sample with
 	 * fraction unless you need exact precision.
-	 * <p/>
+	 * </p>
 	 *
 	 * @param withReplacement Whether element can be selected more than once.
 	 * @param numSample       The expected sample size.
@@ -225,7 +225,7 @@ public final class DataSetUtils {
 	 * <p>
 	 * <strong>NOTE:</strong> Sample with fixed size is not as efficient as sample with fraction, use sample with
 	 * fraction unless you need exact precision.
-	 * <p/>
+	 * </p>
 	 *
 	 * @param withReplacement Whether element can be selected more than once.
 	 * @param numSample       The expected sample size.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java
----------------------------------------------------------------------
diff --git a/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java b/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java
index 4dae630..1ad4f41 100644
--- a/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java
+++ b/flink-java8/src/main/java/org/apache/flink/examples/java8/relational/TPCHQuery10.java
@@ -35,7 +35,7 @@ import org.apache.flink.api.java.tuple.Tuple6;
  * This program implements the following SQL equivalent:
  * 
  * <p>
- * <code><pre>
+ * <pre>{@code
  * SELECT 
  *        c_custkey,
  *        c_name, 
@@ -60,7 +60,7 @@ import org.apache.flink.api.java.tuple.Tuple6;
  *        c_acctbal, 
  *        n_name, 
  *        c_address
- * </pre></code>
+ * }</pre>
  *        
  * <p>
  * Compared to the original TPC-H query this version does not print 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/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 68bfb5b..4f603f7 100755
--- 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
@@ -352,10 +352,10 @@ public class Graph<K, VV, EV> {
 	* @return An instance of {@link org.apache.flink.graph.GraphCsvReader}, 
 	* on which calling methods to specify types of the Vertex ID, Vertex value and Edge value returns a Graph.
 	* 
-	* @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
-	* {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
+	* @see org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)
+	* @see org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)
+	* @see org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)
+	* @see org.apache.flink.graph.GraphCsvReader#keyType(Class)
 	*/
 	public static GraphCsvReader fromCsvReader(String verticesPath, String edgesPath, ExecutionEnvironment context) {
 		return new GraphCsvReader(verticesPath, edgesPath, context);
@@ -369,10 +369,10 @@ public class Graph<K, VV, EV> {
 	* @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
 	* on which calling methods to specify types of the Vertex ID, Vertex value and Edge value returns a Graph.
 	* 
-	* @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
-	* {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
-	* {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
+	* @see org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)
+	* @see org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)
+	* @see org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)
+	* @see org.apache.flink.graph.GraphCsvReader#keyType(Class)
 	*/
 	public static GraphCsvReader fromCsvReader(String edgesPath, ExecutionEnvironment context) {
 		return new GraphCsvReader(edgesPath, context);
@@ -389,10 +389,10 @@ public class Graph<K, VV, EV> {
 	 * @return An instance of {@link org.apache.flink.graph.GraphCsvReader},
 	 * on which calling methods to specify types of the Vertex ID, Vertex Value and Edge value returns a Graph.
 	 * 
-	 * @see {@link org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)},
-	 * {@link org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)},
-	 * {@link org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)} and
-	 * {@link org.apache.flink.graph.GraphCsvReader#keyType(Class)}.
+	 * @see org.apache.flink.graph.GraphCsvReader#types(Class, Class, Class)
+	 * @see org.apache.flink.graph.GraphCsvReader#vertexTypes(Class, Class)
+	 * @see org.apache.flink.graph.GraphCsvReader#edgeTypes(Class, Class)
+	 * @see org.apache.flink.graph.GraphCsvReader#keyType(Class)
 	 */
 	public static <K, VV> GraphCsvReader fromCsvReader(String edgesPath,
 			final MapFunction<K, VV> vertexValueInitializer, ExecutionEnvironment context) {
@@ -821,7 +821,7 @@ public class Graph<K, VV, EV> {
 	/**
 	 * Return the out-degree of all vertices in the graph
 	 * 
-	 * @return A DataSet of Tuple2<vertexId, outDegree>
+	 * @return A DataSet of {@code Tuple2<vertexId, outDegree>}
 	 */
 	public DataSet<Tuple2<K, Long>> outDegrees() {
 
@@ -851,7 +851,7 @@ public class Graph<K, VV, EV> {
 	/**
 	 * Return the in-degree of all vertices in the graph
 	 * 
-	 * @return A DataSet of Tuple2<vertexId, inDegree>
+	 * @return A DataSet of {@code Tuple2<vertexId, inDegree>}
 	 */
 	public DataSet<Tuple2<K, Long>> inDegrees() {
 
@@ -861,7 +861,7 @@ public class Graph<K, VV, EV> {
 	/**
 	 * Return the degree of all vertices in the graph
 	 * 
-	 * @return A DataSet of Tuple2<vertexId, degree>
+	 * @return A DataSet of {@code Tuple2<vertexId, degree>}
 	 */
 	public DataSet<Tuple2<K, Long>> getDegrees() {
 		return outDegrees().union(inDegrees()).groupBy(0).sum(1);

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
index 2a10bd1..712be3e 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/EuclideanGraphWeighing.java
@@ -35,7 +35,7 @@ import java.io.Serializable;
 
 /**
  * This example shows how to use Gelly's {@link Graph#getTriplets()} and
- * {@link Graph#joinWithEdges(DataSet, MapFunction)} methods.
+ * {@link Graph#joinWithEdges(DataSet, EdgeJoinFunction)} methods.
  * 
  * Given a directed, unweighted graph, with vertex values representing points in a plan,
  * return a weighted graph where the edge weights are equal to the Euclidean distance between the
@@ -51,7 +51,6 @@ import java.io.Serializable;
  * 	Edges themselves are separated by newlines.
  * 	For example: <code>1,2\n1,3\n</code> defines two edges 1-2 and 1-3.
  * </ul>
- * </p>
  *
  * Usage <code>EuclideanGraphWeighing &lt;vertex path&gt; &lt;edge path&gt; &lt;result path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
index b808e76..117f7d1 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/GraphMetrics.java
@@ -42,7 +42,7 @@ import org.apache.flink.types.NullValue;
  *
  * The input file is expected to contain one edge per line,
  * with long IDs and no values, in the following format:
- * "<sourceVertexID>\t<targetVertexID>".
+ * "&lt;sourceVertexID&gt;\t&lt;targetVertexID&gt;".
  * If no arguments are provided, the example runs with a random graph of 100 vertices.
  *
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
index c03937d..947f343 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/IncrementalSSSP.java
@@ -52,7 +52,7 @@ import org.apache.flink.graph.spargel.VertexUpdateFunction;
  * The edge is simply removed from the graph.
  * - If the removed edge is an SP-edge, then all nodes, whose shortest path contains the removed edge,
  * potentially require re-computation.
- * When the edge <u, v> is removed, v checks if it has another out-going SP-edge.
+ * When the edge {@code <u, v>} is removed, v checks if it has another out-going SP-edge.
  * If yes, no further computation is required.
  * If v has no other out-going SP-edge, it invalidates its current value, by setting it to INF.
  * Then, it informs all its SP-in-neighbors by sending them an INVALIDATE message.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
index 297dce2..24244c8 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/example/MusicProfiles.java
@@ -45,10 +45,10 @@ import org.apache.flink.util.Collector;
 
 /**
  * This example demonstrates how to mix the DataSet Flink API with the Gelly API.
- * The input is a set <userId - songId - playCount> triplets and
+ * The input is a set &lt;userId - songId - playCount&gt; triplets and
  * a set of bad records, i.e. song ids that should not be trusted.
  * Initially, we use the DataSet API to filter out the bad records.
- * Then, we use Gelly to create a user -> song weighted bipartite graph and compute
+ * Then, we use Gelly to create a user -&gt; song weighted bipartite graph and compute
  * the top song (most listened) per user.
  * Then, we use the DataSet API again, to create a user-user similarity graph,
  * based on common songs, where users that are listeners of the same song
@@ -58,11 +58,11 @@ import org.apache.flink.util.Collector;
  * the similarity graph.
  *
  * The triplets input is expected to be given as one triplet per line,
- * in the following format: "<userID>\t<songID>\t<playcount>".
+ * in the following format: "&lt;userID&gt;\t&lt;songID&gt;\t&lt;playcount&gt;".
  *
  * The mismatches input file is expected to contain one mismatch record per line,
  * in the following format:
- * "ERROR: <songID trackID> song_title"
+ * "ERROR: &lt;songID trackID&gt; song_title"
  *
  * If no arguments are provided, the example runs with default data from {@link MusicProfilesData}.
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
index 5a8e97a..e9add7c 100755
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/ApplyFunction.java
@@ -45,7 +45,7 @@ public abstract class ApplyFunction<K, VV, M> implements Serializable {
 
 	/**
 	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
+	 * @return the number of vertices if the {@link org.apache.flink.graph.IterationConfiguration#setOptNumVertices(boolean)}
 	 * option has been set; -1 otherwise.
 	 */
 	public long getNumberOfVertices() {
@@ -80,15 +80,11 @@ public abstract class ApplyFunction<K, VV, M> implements Serializable {
 
 	/**
 	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
 	 */
 	public void preSuperstep() {}
 
 	/**
 	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
 	 */
 	public void postSuperstep() {}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
index 563b20e..d914f2a 100755
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/GatherFunction.java
@@ -43,7 +43,7 @@ public abstract class GatherFunction<VV, EV, M> implements Serializable {
 
 	/**
 	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
+	 * @return the number of vertices if the {@link org.apache.flink.graph.IterationConfiguration#setOptNumVertices(boolean)}
 	 * option has been set; -1 otherwise.
 	 */
 	public long getNumberOfVertices() {
@@ -69,15 +69,11 @@ public abstract class GatherFunction<VV, EV, M> implements Serializable {
 
 	/**
 	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
 	 */
 	public void preSuperstep() {}
 
 	/**
 	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
 	 */
 	public void postSuperstep() {}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
index 7fa1ed2..4c970fb 100755
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/Neighbor.java
@@ -21,8 +21,8 @@ package org.apache.flink.graph.gsa;
 import org.apache.flink.api.java.tuple.Tuple2;
 
 /**
- * This class represents a <sourceVertex, edge> pair
- * This is a wrapper around Tuple2<VV, EV> for convenience in the GatherFunction
+ * This class represents a {@code <sourceVertex, edge>} pair
+ * This is a wrapper around {@code Tuple2<VV, EV>} for convenience in the GatherFunction
  * @param <VV> the vertex value type
  * @param <EV> the edge value type
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
index f27e275..68e8d27 100755
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/gsa/SumFunction.java
@@ -43,7 +43,7 @@ public abstract class SumFunction<VV, EV, M> implements Serializable {
 
 	/**
 	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfigurationion#setOptNumVertices(boolean)}
+	 * @return the number of vertices if the {@link org.apache.flink.graph.IterationConfiguration#setOptNumVertices(boolean)}
 	 * option has been set; -1 otherwise.
 	 */
 	public long getNumberOfVertices() {
@@ -69,15 +69,11 @@ public abstract class SumFunction<VV, EV, M> implements Serializable {
 
 	/**
 	 * This method is executed once per superstep before the vertex update function is invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the pre-superstep phase cause the superstep to fail.
 	 */
 	public void preSuperstep() {}
 
 	/**
 	 * This method is executed once per superstep after the vertex update function has been invoked for each vertex.
-	 *
-	 * @throws Exception Exceptions in the post-superstep phase cause the superstep to fail.
 	 */
 	public void postSuperstep() {}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/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 745c103..a190cc5 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
@@ -41,7 +41,7 @@ import org.apache.flink.types.NullValue;
  * 
  * The result is a DataSet of vertices, where the vertex value corresponds to the assigned component ID.
  * 
- * @see {@link org.apache.flink.graph.library.GSAConnectedComponents}
+ * @see org.apache.flink.graph.library.GSAConnectedComponents
  */
 @SuppressWarnings("serial")
 public class ConnectedComponents<K, EV> implements GraphAlgorithm<K, Long, EV, DataSet<Vertex<K, Long>>> {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/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 4269517..a44ba14 100755
--- 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
@@ -34,7 +34,7 @@ import org.apache.flink.types.NullValue;
  * This implementation assumes that the vertices of the input Graph are initialized with unique, Long component IDs.
  * The result is a DataSet of vertices, where the vertex value corresponds to the assigned component ID.
  * 
- * @see {@link org.apache.flink.graph.library.ConnectedComponents}
+ * @see org.apache.flink.graph.library.ConnectedComponents
  */
 public class GSAConnectedComponents<K, EV> implements GraphAlgorithm<K, Long, EV, DataSet<Vertex<K, Long>>> {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
index 43a5e5c..99624ca 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java
@@ -61,7 +61,7 @@ public class GSAPageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet
 	/**
 	 * Creates an instance of the GSA PageRank algorithm.
 	 * If the number of vertices of the input graph is known,
-	 * use the {@link GSAPageRank#GSAPageRank(double, long)} constructor instead.
+	 * use the {@link GSAPageRank#GSAPageRank(double, int)} constructor instead.
 	 * 
 	 * The implementation assumes that each page has at least one incoming and one outgoing link.
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
index dfbe14b..935058d 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/PageRank.java
@@ -62,7 +62,7 @@ public class PageRank<K> implements GraphAlgorithm<K, Double, Double, DataSet<Ve
 	/**
 	 * Creates an instance of the PageRank algorithm.
 	 * If the number of vertices of the input graph is known,
-	 * use the {@link PageRank#PageRank(double, long)} constructor instead.
+	 * use the {@link PageRank#PageRank(double, int)} constructor instead.
 	 * 
 	 * The implementation assumes that each page has at least one incoming and one outgoing link.
 	 * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
index 271db86..f29fc9d 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/MessagingFunction.java
@@ -53,7 +53,7 @@ public abstract class MessagingFunction<K, VV, Message, EV> implements Serializa
 
 	/**
 	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfiguration#setOptNumVertices(boolean)}
+	 * @return the number of vertices if the {@link org.apache.flink.graph.IterationConfiguration#setOptNumVertices(boolean)}
 	 * option has been set; -1 otherwise.
 	 */
 	public long getNumberOfVertices() {
@@ -314,8 +314,7 @@ public abstract class MessagingFunction<K, VV, Message, EV> implements Serializa
 
 	/**
 	 * Retrieves the vertex in-degree (number of in-coming edges).
-	 * @return The in-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
+	 * @return The in-degree of this vertex
 	 */
 	public long getInDegree() {
 		return inDegree;
@@ -327,8 +326,7 @@ public abstract class MessagingFunction<K, VV, Message, EV> implements Serializa
 
 	/**
 	 * Retrieve the vertex out-degree (number of out-going edges).
-	 * @return The out-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
+	 * @return The out-degree of this vertex
 	 */
 	public long getOutDegree() {
 		return outDegree;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
index fdc39ff..be48f9c 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexCentricIteration.java
@@ -46,7 +46,7 @@ import com.google.common.base.Preconditions;
 
 /**
  * This class represents iterative graph computations, programmed in a vertex-centric perspective.
- * It is a special case of <i>Bulk Synchronous Parallel<i> computation. The paradigm has also been
+ * It is a special case of <i>Bulk Synchronous Parallel</i> computation. The paradigm has also been
  * implemented by Google's <i>Pregel</i> system and by <i>Apache Giraph</i>.
  * <p>
  * Vertex centric algorithms operate on graphs, which are defined through vertices and edges. The 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
index 248925b..857cef5 100644
--- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
+++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/spargel/VertexUpdateFunction.java
@@ -30,12 +30,12 @@ import org.apache.flink.util.Collector;
 
 /**
  * This class must be extended by functions that compute the state of the vertex depending on the old state and the
- * incoming messages. The central method is {@link #updateVertex(Comparable, Object, MessageIterator)}, which is
+ * incoming messages. The central method is {@link #updateVertex(Vertex, MessageIterator)}, which is
  * invoked once per vertex per superstep.
  * 
- * <K> The vertex key type.
- * <VV> The vertex value type.
- * <Message> The message type.
+ * {@code <K>} The vertex key type.
+ * {@code <VV>} The vertex value type.
+ * {@code <Message>} The message type.
  */
 public abstract class VertexUpdateFunction<K, VV, Message> implements Serializable {
 
@@ -50,7 +50,7 @@ public abstract class VertexUpdateFunction<K, VV, Message> implements Serializab
 
 	/**
 	 * Retrieves the number of vertices in the graph.
-	 * @return the number of vertices if the {@link IterationConfiguration#setOptNumVertices(boolean)}
+	 * @return the number of vertices if the {@link org.apache.flink.graph.IterationConfiguration#setOptNumVertices(boolean)}
 	 * option has been set; -1 otherwise.
 	 */
 	public long getNumberOfVertices() {
@@ -206,8 +206,7 @@ public abstract class VertexUpdateFunction<K, VV, Message> implements Serializab
 
 	/**
 	 * Retrieves the vertex in-degree (number of in-coming edges).
-	 * @return The in-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
+	 * @return The in-degree of this vertex
 	 */
 	public long getInDegree() {
 		return inDegree;
@@ -219,8 +218,7 @@ public abstract class VertexUpdateFunction<K, VV, Message> implements Serializab
 
 	/**
 	 * Retrieve the vertex out-degree (number of out-going edges).
-	 * @return The out-degree of this vertex if the {@link IterationConfiguration#setOptDegrees(boolean)}
-	 * option has been set; -1 otherwise. 
+	 * @return The out-degree of this vertex
 	 */
 	public long getOutDegree() {
 		return outDegree;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
index 490c304..a3bf995 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/OptimizerNode.java
@@ -50,7 +50,7 @@ import org.apache.flink.util.Visitor;
  * The OptimizerNode is the base class of all nodes in the optimizer DAG. The optimizer DAG is the
  * optimizer's representation of a program, created before the actual optimization (which creates different
  * candidate plans and computes their cost).
- * <p>>
+ * <p>
  * Nodes in the DAG correspond (almost) one-to-one to the operators in a program. The optimizer DAG is constructed
  * to hold the additional information that the optimizer needs:
  * <ul>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java
index c8be5d4..ffa317c 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/OperatorDescriptorSingle.java
@@ -35,7 +35,7 @@ import org.apache.flink.optimizer.plan.SingleInputPlanNode;
  * Abstract base class for Operator descriptions which instantiates the node and sets the driver
  * strategy and the sorting and grouping keys. Returns possible local and global properties and
  * updates them after the operation has been performed.
- * @see org.apache.flink.compiler.dag.SingleInputNode
+ * @see org.apache.flink.optimizer.dag.SingleInputNode
  */
 public abstract class OperatorDescriptorSingle implements AbstractOperatorDescriptor {
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
index b139b62..a4448c6 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java
@@ -186,7 +186,6 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
 	 * Sets the data exchange mode (batch / pipelined) to use for the data
 	 * exchange of this channel.
 	 *
-	 * @return The data exchange mode of this channel.
 	 */
 	public void setDataExchangeMode(DataExchangeMode dataExchangeMode) {
 		this.dataExchangeMode = checkNotNull(dataExchangeMode);

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/SparseKeySchema.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/SparseKeySchema.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/SparseKeySchema.java
index e14888e..1545d6f 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/SparseKeySchema.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/postpass/SparseKeySchema.java
@@ -26,7 +26,7 @@ import java.util.Map.Entry;
 import org.apache.flink.types.Key;
 
 /**
- * Class encapsulating a schema map (int column position -> column type) and a reference counter.
+ * Class encapsulating a schema map (int column position -&gt; column type) and a reference counter.
  */
 public class SparseKeySchema extends AbstractSchema<Class<? extends Key<?>>> {
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
index a2400b5..e6ea8d3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java
@@ -52,7 +52,6 @@ public interface BlobService {
 
 	/**
 	 * Shutdown method which is called to terminate the blob service.
-	 * @throws IOException
 	 */
 	void shutdown();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
index 81159f6..19c65d4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
@@ -149,7 +149,6 @@ public class PendingCheckpoint {
 	
 	/**
 	 * Discards the pending checkpoint, releasing all held resources.
-	 * @throws Exception 
 	 */
 	public void discard(ClassLoader userClassLoader, boolean discardStateHandle) {
 		synchronized (lock) {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
index 9f4a5a7..e3e3256 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
@@ -22,7 +22,7 @@ package org.apache.flink.runtime.execution;
  * An enumeration of all states that a task can be in during its execution.
  * Tasks usually start in the state {@code CREATED} and switch states according to
  * this diagram:
- * <pre>
+ * <pre>{@code
  *
  *     CREATED  -> SCHEDULED -> DEPLOYING -> RUNNING -> FINISHED
  *                     |            |          |
@@ -33,7 +33,7 @@ package org.apache.flink.runtime.execution;
  *                     +-------------------------+
  *
  *                                               ... -> FAILED
- * </pre>
+ * }</pre>
  *
  * <p>It is possible to enter the {@code FAILED} state from any other state.</p>
  *

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
index fa5ad85..1b8b1d9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
@@ -50,7 +50,7 @@ import org.slf4j.LoggerFactory;
 /**
  * The FileCache is used to create the local files for the registered cache files when a task is deployed.
  * The files will be removed when the task is unregistered after a 5 second delay.
- * A given file x will be placed in "<system-tmp-dir>/tmp_<jobID>/".
+ * A given file x will be placed in "{@code <system-tmp-dir>/tmp_<jobID>/}".
  */
 public class FileCache {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java
index 94249de..7d666fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java
@@ -57,7 +57,7 @@ import org.slf4j.LoggerFactory;
  * 
  * <p>Consider a job set up like this:</p>
  * 
- * <pre>
+ * <pre>{@code
  * +-------------- Slot Sharing Group --------------+
  * |                                                |
  * |            +-- Co Location Group --+           |
@@ -66,7 +66,7 @@ import org.slf4j.LoggerFactory;
  * |            |                       |           |
  * |            +-----------------------+           |
  * +------------------------------------------------+
- * </pre>
+ * }</pre>
  * 
  * <p>The slot hierarchy in the slot sharing group will look like the following</p> 
  * 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
index 957052e..8e33cda 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java
@@ -46,7 +46,7 @@ public interface BlockChannelReader<T> extends FileIOChannel {
 	 * such a segment is available, or until an error occurs in the reader, or the reader is closed.
 	 * <p>
 	 * WARNING: If this method is invoked without any segment ever returning (for example, because the
-	 * {@link #readBlock(T)} method has not been invoked appropriately), the method may block
+	 * {@link #readBlock} method has not been invoked appropriately), the method may block
 	 * forever.
 	 * 
 	 * @return The next memory segment from the reader's return queue.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
index ccf065a..d6cfd7e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java
@@ -35,7 +35,7 @@ public interface BlockChannelWriter<T> extends BlockChannelWriterWithCallback<T>
 	 * writer is closed.
 	 * <p>
 	 * NOTE: If this method is invoked without any segment ever returning (for example, because the
-	 * {@link #writeBlock(T)} method has not been invoked accordingly), the method may block
+	 * {@link #writeBlock} method has not been invoked accordingly), the method may block
 	 * forever.
 	 * 
 	 * @return The next memory segment from the writers's return queue.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
index f18c7e4..d49b27e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputGate.java
@@ -32,17 +32,17 @@ import java.io.IOException;
  * <p> As an example, consider a map-reduce program, where the map operator produces data and the
  * reduce operator consumes the produced data.
  *
- * <pre>
+ * <pre>{@code
  * +-----+              +---------------------+              +--------+
  * | Map | = produce => | Intermediate Result | <= consume = | Reduce |
  * +-----+              +---------------------+              +--------+
- * </pre>
+ * }</pre>
  *
  * <p> When deploying such a program in parallel, the intermediate result will be partitioned over its
  * producing parallel subtasks; each of these partitions is furthermore partitioned into one or more
  * subpartitions.
  *
- * <pre>
+ * <pre>{@code
  *                            Intermediate result
  *               +-----------------------------------------+
  *               |                      +----------------+ |              +-----------------------+
@@ -57,7 +57,7 @@ import java.io.IOException;
  * +-------+     | +-------------+  +=> | Subpartition 2 | | <==+======== | Input Gate | Reduce 2 |
  *               |                      +----------------+ |              +-----------------------+
  *               +-----------------------------------------+
- * </pre>
+ * }</pre>
  *
  * <p> In the above example, two map subtasks produce the intermediate result in parallel, resulting
  * in two partitions (Partition 1 and 2). Each of these partitions is further partitioned into two

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
index 896fa9c..47f9228 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java
@@ -66,17 +66,17 @@ import static com.google.common.base.Preconditions.checkState;
  * <p> As an example, consider a map-reduce program, where the map operator produces data and the
  * reduce operator consumes the produced data.
  *
- * <pre>
+ * <pre>{@code
  * +-----+              +---------------------+              +--------+
  * | Map | = produce => | Intermediate Result | <= consume = | Reduce |
  * +-----+              +---------------------+              +--------+
- * </pre>
+ * }</pre>
  *
  * <p> When deploying such a program in parallel, the intermediate result will be partitioned over its
  * producing parallel subtasks; each of these partitions is furthermore partitioned into one or more
  * subpartitions.
  *
- * <pre>
+ * <pre>{@code
  *                            Intermediate result
  *               +-----------------------------------------+
  *               |                      +----------------+ |              +-----------------------+
@@ -91,7 +91,7 @@ import static com.google.common.base.Preconditions.checkState;
  * +-------+     | +-------------+  +=> | Subpartition 2 | | <==+======== | Input Gate | Reduce 2 |
  *               |                      +----------------+ |              +-----------------------+
  *               +-----------------------------------------+
- * </pre>
+ * }</pre>
  *
  * <p> In the above example, two map subtasks produce the intermediate result in parallel, resulting
  * in two partitions (Partition 1 and 2). Each of these partitions is further partitioned into two

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java
index 0d266c2..6836237 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/IterationIntermediateTask.java
@@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 
 /**
- * An intermediate iteration task, which runs a {@link org.apache.flink.runtime.operators.PactDriver} inside.
+ * An intermediate iteration task, which runs a {@link org.apache.flink.runtime.operators.Driver} inside.
  * <p>
  * It will propagate {@link EndOfSuperstepEvent}s and {@link TerminationEvent}s to it's connected tasks. Furthermore
  * intermediate tasks can also update the iteration state, either the workset or the solution set.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index cb99e52..963fc4c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -57,7 +57,7 @@ import scala.concurrent.ExecutionContext;
  * <p>The scheduler supports two scheduling modes:</p>
  * <ul>
  *     <li>Immediate scheduling: A request for a task slot immediately returns a task slot, if one is
- *         available, or throws a {@link NoResourceAvailableException}</li>.
+ *         available, or throws a {@link NoResourceAvailableException}.</li>
  *     <li>Queued Scheduling: A request for a task slot is queued and returns a future that will be
  *         fulfilled as soon as a slot becomes available.</li>
  * </ul>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
index db56227..1b54f20 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
@@ -53,7 +53,7 @@ public abstract class AbstractMergeInnerJoinIterator<T1, T2, O> extends Abstract
 	/**
 	 * Calls the <code>JoinFunction#join()</code> method for all two key-value pairs that share the same key and come
 	 * from different inputs. The output of the <code>join()</code> method is forwarded.
-	 * <p/>
+	 * <p>
 	 * This method first zig-zags between the two sorted inputs in order to find a common
 	 * key, and then calls the join stub with the cross product of the values.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
index 947a56b..0ad3c6d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/BloomFilter.java
@@ -6,9 +6,9 @@
  * 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
- * <p/>
+ * <p>
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ * <p>
  * 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.
@@ -34,11 +34,11 @@ import static com.google.common.base.Preconditions.checkArgument;
  * During the creation of bloom filter expected number of entries must be specified. If the number
  * of insertions exceed the specified initial number of entries then false positive probability will
  * increase accordingly.
- * <p/>
+ * <p>
  * Internally, this implementation of bloom filter uses MemorySegment to store BitSet, BloomFilter and
  * BitSet are designed to be able to switch between different MemorySegments, so that Flink can share
  * the same BloomFilter/BitSet object instance for different bloom filters.
- * <p/>
+ * <p>
  * Part of this class refers to the implementation from Apache Hive project
  * https://github.com/apache/hive/blob/master/common/src/java/org/apache/hive/common/util/BloomFilter.java
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
index 181d6d6..a7a3458 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java
@@ -113,7 +113,7 @@ public final class WebMonitorUtils {
 	/**
 	 * Starts the web runtime monitor. Because the actual implementation of the runtime monitor is
 	 * in another project, we load the runtime monitor dynamically.
-	 * <p/>
+	 * <p>
 	 * Because failure to start the web runtime monitor is not considered fatal, this method does
 	 * not throw any exceptions, but only logs them.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
index 3f78898..d7ef904 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/yarn/AbstractFlinkYarnCluster.java
@@ -100,7 +100,9 @@ public abstract class AbstractFlinkYarnCluster {
 
 	/**
 	 * Disconnect from the ApplicationMaster without stopping the session
-	 * (therefore, use the {@see shutdown()} method.
+	 * (therefore, use the {@link AbstractFlinkYarnCluster#shutdown(boolean)} method.
+	 *
+	 * @see AbstractFlinkYarnCluster#shutdown(boolean)
 	 */
 	public abstract void disconnect();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
index d18cace..36fb849 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/StateStorageHelper.java
@@ -23,7 +23,7 @@ import org.apache.flink.runtime.state.StateHandle;
 import java.io.Serializable;
 
 /**
- * State storage helper which is used by {@ZooKeeperStateHandleStore} to persiste state before
+ * State storage helper which is used by {@link ZooKeeperStateHandleStore} to persiste state before
  * the state handle is written to ZooKeeper.
  *
  * @param <T>

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
index 6073a39..1d89d64 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java
@@ -50,13 +50,13 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * <p>ZooKeeper holds the ground truth about state handles, i.e. the following holds:
  *
  * <pre>
- * State handle in ZooKeeper => State handle exists
+ * State handle in ZooKeeper =&gt; State handle exists
  * </pre>
  *
  * But not:
  *
  * <pre>
- * State handle exists => State handle in ZooKeeper
+ * State handle exists =&gt; State handle in ZooKeeper
  * </pre>
  *
  * There can be lingering state handles when failures happen during operation. They
@@ -316,7 +316,7 @@ public class ZooKeeperStateHandleStore<T extends Serializable> {
 	/**
 	 * Removes a state handle from ZooKeeper.
 	 *
-	 * <p><stong>Important</stong>: this does not discard the state handle. If you want to
+	 * <p><strong>Important</strong>: this does not discard the state handle. If you want to
 	 * discard the state handle call {@link #removeAndDiscardState(String)}.
 	 *
 	 * @param pathInZooKeeper Path of state handle to remove (expected to start with a '/')
@@ -331,7 +331,7 @@ public class ZooKeeperStateHandleStore<T extends Serializable> {
 	/**
 	 * Removes a state handle from ZooKeeper asynchronously.
 	 *
-	 * <p><stong>Important</stong>: this does not discard the state handle. If you want to
+	 * <p><strong>Important</strong>: this does not discard the state handle. If you want to
 	 * discard the state handle call {@link #removeAndDiscardState(String)}.
 	 *
 	 * @param pathInZooKeeper Path of state handle to remove (expected to start with a '/')

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
index 4d81daf..83ab23d 100644
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
+++ b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java
@@ -111,7 +111,7 @@ public final class HadoopMapFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT>
 	
 	/**
 	 * Custom serialization methods.
-	 *  @see http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html
+	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
 	 */
 	private void writeObject(final ObjectOutputStream out) throws IOException {
 		out.writeObject(mapper.getClass());

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
index f00859f..97b9768 100644
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
+++ b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
@@ -138,7 +138,7 @@ public final class HadoopReduceCombineFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT>
 
 	/**
 	 * Custom serialization methods.
-	 *  @see http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html
+	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
 	 */
 	private void writeObject(final ObjectOutputStream out) throws IOException {
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
index 6943421..1c47696 100644
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
+++ b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
@@ -118,8 +118,8 @@ public final class HadoopReduceFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT>
 	}
 
 	/**
-	 * Custom serialization methods.
-	 *  @see http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html
+	 * Custom serialization methods
+	 * @see <a href="http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html">http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html</a>
 	 */
 	private void writeObject(final ObjectOutputStream out) throws IOException {
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/JarHelper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/JarHelper.java b/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/JarHelper.java
index e085c25..83fb342 100644
--- a/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/JarHelper.java
+++ b/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/JarHelper.java
@@ -37,7 +37,7 @@ import java.util.jar.JarInputStream;
  *
  * Copied from http://grepcode.com/file_/repo1.maven.org/maven2/org.apache.xmlbeans/xmlbeans/2.4.0/org/apache/xmlbeans/impl/common/JarHelper.java/?v=source
  *
- * @author Patrick Calahan <pc...@bea.com>
+ * @author Patrick Calahan <a href="mailto:pcal@bea.com">pcal@bea.com</a>
  */
 public class JarHelper
 {

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-staging/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java b/flink-staging/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
index 6f9f0a3..97113bb 100644
--- a/flink-staging/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
+++ b/flink-staging/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
@@ -31,7 +31,7 @@
  * <p>
  * Example:
  *
- * <code>
+ * <pre>{@code
  * ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment();
  *
  * DataSet<WC> input = env.fromElements(
@@ -50,11 +50,11 @@
  *
  * result.print();
  * env.execute();
- * </code>
+ * }</pre>
  *
  * <p>
  * As seen above, a {@link org.apache.flink.api.table.Table} can be converted back to the
- * underlying API representation using {@link org.apache.flink.api.java.table.TableEnvironment.toSet}
- * or {@link org.apache.flink.api.java.table.TableEnvironment.toStream}.
+ * underlying API representation using {@link org.apache.flink.api.java.table.TableEnvironment#toDataSet(org.apache.flink.api.table.Table, java.lang.Class)}
+ * or {@link org.apache.flink.api.java.table.TableEnvironment#toDataStream(org.apache.flink.api.table.Table, java.lang.Class)}}.
  */
 package org.apache.flink.api.java.table;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
index 8066b3c..c7ae1cc 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
@@ -79,7 +79,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
  * <ul>
  *     <li>{@link FetcherType#NEW_HIGH_LEVEL}: A fetcher based on the new Kafka consumer API.
  *         This fetcher is generally more robust, but works only with later versions of
- *         Kafka (> 0.8.2).</li>
+ *         Kafka (&gt; 0.8.2).</li>
  *         
  *     <li>{@link FetcherType#LEGACY_LOW_LEVEL}: A fetcher based on the old low-level consumer API.
  *         This fetcher is works also with older versions of Kafka (0.8.1). The fetcher interprets

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
index 346a7d5..7ab7290 100644
--- a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
@@ -28,24 +28,24 @@ import java.io.Serializable;
  * 	# More Flink partitions than kafka partitions
  * <pre>
  * 		Flink Sinks:		Kafka Partitions
- * 			1	---------------->	1
+ * 			1	----------------&gt;	1
  * 			2   --------------/
  * 			3   -------------/
  * 			4	------------/
  * </pre>
- * 	--> Some (or all) kafka partitions contain the output of more than one flink partition
+ * Some (or all) kafka partitions contain the output of more than one flink partition
  *
  *# Fewer Flink partitions than Kafka
  * <pre>
  * 		Flink Sinks:		Kafka Partitions
- * 			1	---------------->	1
- * 			2	---------------->	2
- * 									3
- * 									4
- * 									5
+ * 			1	----------------&gt;	1
+ * 			2	----------------&gt;	2
+ * 										3
+ * 										4
+ * 										5
  * </pre>
  *
- *  --> Not all Kafka partitions contain data
+ *  Not all Kafka partitions contain data
  *  To avoid such an unbalanced partitioning, use a round-robin kafka partitioner. (note that this will
  *  cause a lot of network connections between all the Flink instances and all the Kafka brokers
  *

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
index 2cf66b9..52ec896 100644
--- a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
@@ -32,10 +32,11 @@ import java.util.List;
 import java.util.Random;
 
 /**
- * Example illustrating iterations in Flink streaming. <p/> <p> The program sums up random numbers and counts additions
+ * Example illustrating iterations in Flink streaming.
+ * <p> The program sums up random numbers and counts additions
  * it performs to reach a specific threshold in an iterative streaming fashion. </p>
- * <p/>
- * <p/>
+ *
+ * <p>
  * This example shows how to use: <ul> <li>streaming iterations, <li>buffer timeout to enhance latency, <li>directed
  * outputs. </ul>
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index ce227e4..32cf430 100644
--- a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -40,7 +40,7 @@ import java.util.concurrent.TimeUnit;
  * incremental Alternating Least Squares model while also providing the
  * predictions.
  *
- * <p/>
+ * <p>
  * This example shows how to use:
  * <ul>
  *   <li>Connected streams
@@ -188,7 +188,6 @@ public class IncrementalLearningSkeleton {
 	/**
 	 * Creates newData using the model produced in batch-processing and the
 	 * up-to-date partial model.
-	 * <p/>
 	 * <p>
 	 * By defaults emits the Integer 0 for every newData and the Integer 1
 	 * for every model update.

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
index 17add2c..cecabdd 100644
--- a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
@@ -26,19 +26,16 @@ import org.apache.flink.streaming.examples.wordcount.WordCount.Tokenizer;
  * This example shows an implementation of WordCount with data from a text
  * socket. To run the example make sure that the service providing the text data
  * is already up and running.
- * <p/>
- * <p/>
+ * <p>
  * To start an example socket text stream on your local machine run netcat from
  * a command line: <code>nc -lk 9999</code>, where the parameter specifies the
  * port number.
- * <p/>
- * <p/>
- * <p/>
+ * </p>
+ * <p>
  * Usage:
  * <code>SocketTextStreamWordCount &lt;hostname&gt; &lt;port&gt; &lt;result path&gt;</code>
- * <br>
- * <p/>
- * <p/>
+ * </p>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>use StreamExecutionEnvironment.socketTextStream

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
index c2477b5..06872f0 100644
--- a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
@@ -31,16 +31,15 @@ import java.util.StringTokenizer;
 /**
  * Implements the "TwitterStream" program that computes a most used word
  * occurrence over JSON files in a streaming fashion.
- * <p/>
- * <p/>
+ * <p>
  * The input is a JSON text file with lines separated by newline characters.
- * <p/>
- * <p/>
+ * </p>
+ * <p>
  * Usage: <code>TwitterStream &lt;text path&gt;</code><br>
  * If no parameters are provided, the program is run with default data from
  * {@link TwitterStreamData}.
- * <p/>
- * <p/>
+ * </p>
+ * <p>
  * This example shows how to:
  * <ul>
  * <li>acquire external data,
@@ -88,13 +87,11 @@ public class TwitterStream {
 
 	/**
 	 * Makes sentences from English tweets.
-	 * <p/>
 	 * <p>
 	 * Implements a string tokenizer that splits sentences into words as a
 	 * user-defined FlatMapFunction. The function takes a line (String) and
-	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
-	 * Integer>).
-	 * </p>
+	 * splits it into multiple pairs in the form of "(word,1)" ({@code Tuple2<String,
+	 * Integer>}).
 	 */
 	public static class SelectEnglishAndTokenizeFlatMap extends JSONParseFlatMap<String, Tuple2<String, Integer>> {
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
index 591ef51..9b0b63c 100644
--- a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
@@ -126,8 +126,8 @@ public class PojoExample {
 	/**
 	 * Implements the string tokenizer that splits sentences into words as a
 	 * user-defined FlatMapFunction. The function takes a line (String) and
-	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
-	 * Integer>).
+	 * splits it into multiple pairs in the form of "(word,1)" ({@code Tuple2<String,
+	 * Integer>}).
 	 */
 	public static final class Tokenizer implements FlatMapFunction<String, Word> {
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
index a594c94..a76671e 100644
--- a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
@@ -87,8 +87,8 @@ public class WordCount {
 	/**
 	 * Implements the string tokenizer that splits sentences into words as a
 	 * user-defined FlatMapFunction. The function takes a line (String) and
-	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
-	 * Integer>).
+	 * splits it into multiple pairs in the form of "(word,1)" ({@code Tuple2<String,
+	 * Integer>}).
 	 */
 	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
 		private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/flink/blob/680b5a90/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 176a07f..309eb05 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -598,7 +598,7 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Initiates a Project transformation on a {@link Tuple} {@link DataStream}.<br/>
+	 * Initiates a Project transformation on a {@link Tuple} {@link DataStream}.<br>
 	 * <b>Note: Only Tuple DataStreams can be projected.</b>
 	 *
 	 * <p>


[04/13] flink git commit: [FLINK-2900] [hadoop-compat] Remove Record API code from Hadoop Compat module

Posted by fh...@apache.org.
[FLINK-2900] [hadoop-compat] Remove Record API code from Hadoop Compat module

This closes #1293


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

Branch: refs/heads/master
Commit: 2ff04baa649831fe66a7fa1f3d5b4d7adf632261
Parents: a8f1be9
Author: Fabian Hueske <fh...@apache.org>
Authored: Thu Oct 22 21:12:15 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 10:33:55 2015 +0100

----------------------------------------------------------------------
 .../mapred/record/HadoopDataSink.java           |  98 ----------
 .../mapred/record/HadoopDataSource.java         |  82 --------
 .../mapred/record/HadoopRecordInputFormat.java  | 174 ----------------
 .../mapred/record/HadoopRecordOutputFormat.java | 156 ---------------
 .../datatypes/DefaultFlinkTypeConverter.java    |  95 ---------
 .../datatypes/DefaultHadoopTypeConverter.java   |  83 --------
 .../record/datatypes/FlinkTypeConverter.java    |  43 ----
 .../datatypes/HadoopFileOutputCommitter.java    | 196 -------------------
 .../record/datatypes/HadoopTypeConverter.java   |  42 ----
 .../datatypes/WritableComparableWrapper.java    |  40 ----
 .../record/datatypes/WritableWrapper.java       |  71 -------
 .../datatypes/WritableWrapperConverter.java     |  45 -----
 .../mapred/record/example/WordCount.java        | 184 -----------------
 .../example/WordCountWithOutputFormat.java      | 173 ----------------
 .../record/HadoopRecordInputOutputITCase.java   |  54 -----
 15 files changed, 1536 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSink.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSink.java
deleted file mode 100644
index 3b8064f..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSink.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred.record;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.operators.Operator;
-import org.apache.flink.api.java.record.operators.GenericDataSink;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.DefaultFlinkTypeConverter;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.FlinkTypeConverter;
-import org.apache.flink.types.Record;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputFormat;
-
-/**
- * The HadoopDataSink is a generic wrapper for all Hadoop OutputFormats.
- *
- * Example usage:
- * <pre>
- * 		HadoopDataSink out = new HadoopDataSink(new org.apache.hadoop.mapred.TextOutputFormat<Text, IntWritable>(), new JobConf(), "Hadoop TextOutputFormat",reducer, Text.class,IntWritable.class);
- *		org.apache.hadoop.mapred.TextOutputFormat.setOutputPath(out.getJobConf(), new Path(output));
- * </pre>
- *
- * Note that it is possible to provide custom data type converter.
- *
- * The HadoopDataSink provides a default converter: {@link org.apache.flink.hadoopcompatibility.mapred.record.datatypes.DefaultFlinkTypeConverter}
- **/
-public class HadoopDataSink<K,V> extends GenericDataSink {
-
-	private static String DEFAULT_NAME = "<Unnamed Hadoop Data Sink>";
-
-	private JobConf jobConf;
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, JobConf jobConf, String name, Operator<Record> input, FlinkTypeConverter<K,V> conv, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, jobConf, name, Collections.<Operator<Record>>singletonList(input), conv, keyClass, valueClass);
-	}
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, JobConf jobConf, String name, Operator<Record> input, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, jobConf, name, input, new DefaultFlinkTypeConverter<K, V>(keyClass, valueClass), keyClass, valueClass);
-	}
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, JobConf jobConf, Operator<Record> input, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, jobConf, DEFAULT_NAME, input, new DefaultFlinkTypeConverter<K, V>(keyClass, valueClass), keyClass, valueClass);
-	}
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, Operator<Record> input, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, new JobConf(), DEFAULT_NAME, input, new DefaultFlinkTypeConverter<K, V>(keyClass, valueClass), keyClass, valueClass);
-	}
-
-
-
-	@SuppressWarnings("deprecation")
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, JobConf jobConf, String name, List<Operator<Record>> input, FlinkTypeConverter<K,V> conv, Class<K> keyClass, Class<V> valueClass) {
-		super(new HadoopRecordOutputFormat<K,V>(hadoopFormat, jobConf, conv),input, name);
-		
-		if (hadoopFormat == null || jobConf == null) {
-			throw new NullPointerException();
-		}
-		
-		this.name = name;
-		this.jobConf = jobConf;
-		jobConf.setOutputKeyClass(keyClass);
-		jobConf.setOutputValueClass(valueClass);
-	}
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, JobConf jobConf, String name, List<Operator<Record>> input, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, jobConf, name, input, new DefaultFlinkTypeConverter<K, V>(keyClass, valueClass), keyClass, valueClass);
-	}
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, JobConf jobConf, List<Operator<Record>> input, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, jobConf, DEFAULT_NAME, input, new DefaultFlinkTypeConverter<K, V>(keyClass, valueClass), keyClass, valueClass);
-	}
-
-	public HadoopDataSink(OutputFormat<K,V> hadoopFormat, List<Operator<Record>> input, Class<K> keyClass, Class<V> valueClass) {
-		this(hadoopFormat, new JobConf(), DEFAULT_NAME, input, new DefaultFlinkTypeConverter<K, V>(keyClass, valueClass), keyClass, valueClass);
-	}
-
-	public JobConf getJobConf() {
-		return this.jobConf;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSource.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSource.java
deleted file mode 100644
index 508f069..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopDataSource.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred.record;
-
-
-import org.apache.flink.api.java.record.operators.GenericDataSource;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.DefaultHadoopTypeConverter;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.HadoopTypeConverter;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.JobConf;
-
-/**
- * The HadoopDataSource is a generic wrapper for all Hadoop InputFormats.
- * 
- * Example usage:
- * <pre>
- * 		HadoopDataSource source = new HadoopDataSource(new org.apache.hadoop.mapred.TextInputFormat(), new JobConf(), "Input Lines");
- *		org.apache.hadoop.mapred.TextInputFormat.addInputPath(source.getJobConf(), new Path(dataInput));
- * </pre>
- * 
- * Note that it is possible to provide custom data type converter.
- * 
- * The HadoopDataSource provides two different standard converters:
- * * WritableWrapperConverter: Converts Hadoop Types to a record that contains a WritableComparableWrapper (key) and a WritableWrapper
- * * DefaultHadoopTypeConverter: Converts the standard hadoop types (longWritable, Text) to Flinks's {@link org.apache.flink.types.Value} types.
- */
-public class HadoopDataSource<K,V> extends GenericDataSource<HadoopRecordInputFormat<K,V>> {
-
-	private static String DEFAULT_NAME = "<Unnamed Hadoop Data Source>";
-	
-	private JobConf jobConf;
-	
-	/**
-	 * 
-	 * @param hadoopFormat Implementation of a Hadoop input format
-	 * @param jobConf JobConf object (Hadoop)
-	 * @param name Name of the DataSource
-	 * @param conv Definition of a custom type converter {@link DefaultHadoopTypeConverter}.
-	 */
-	public HadoopDataSource(InputFormat<K,V> hadoopFormat, JobConf jobConf, String name, HadoopTypeConverter<K,V> conv) {
-		super(new HadoopRecordInputFormat<K,V>(hadoopFormat, jobConf, conv),name);
-		
-		if (hadoopFormat == null || jobConf == null || conv == null) {
-			throw new NullPointerException();
-		}
-		
-		this.name = name;
-		this.jobConf = jobConf;
-	}
-	
-	public HadoopDataSource(InputFormat<K,V> hadoopFormat, JobConf jobConf, String name) {
-		this(hadoopFormat, jobConf, name, new DefaultHadoopTypeConverter<K,V>() );
-	}
-	public HadoopDataSource(InputFormat<K,V> hadoopFormat, JobConf jobConf) {
-		this(hadoopFormat, jobConf, DEFAULT_NAME);
-	}
-	
-	public HadoopDataSource(InputFormat<K,V> hadoopFormat) {
-		this(hadoopFormat, new JobConf(), DEFAULT_NAME);
-	}
-
-	public JobConf getJobConf() {
-		return this.jobConf;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
deleted file mode 100644
index edcc43b..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-
-import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.HadoopTypeConverter;
-import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopInputSplit;
-import org.apache.flink.types.Record;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.util.ReflectionUtils;
-
-public class HadoopRecordInputFormat<K, V> implements InputFormat<Record, HadoopInputSplit> {
-
-	private static final long serialVersionUID = 1L;
-
-	public org.apache.hadoop.mapred.InputFormat<K, V> hadoopInputFormat;
-	public HadoopTypeConverter<K,V> converter;
-	private String hadoopInputFormatName;
-	public JobConf jobConf;
-	public transient K key;
-	public transient V value;
-	public RecordReader<K, V> recordReader;
-	private boolean fetched = false;
-	private boolean hasNext;
-		
-	public HadoopRecordInputFormat() {
-		super();
-	}
-	
-	public HadoopRecordInputFormat(org.apache.hadoop.mapred.InputFormat<K,V> hadoopInputFormat, JobConf job, HadoopTypeConverter<K,V> conv) {
-		super();
-		this.hadoopInputFormat = hadoopInputFormat;
-		this.hadoopInputFormatName = hadoopInputFormat.getClass().getName();
-		this.converter = conv;
-		HadoopUtils.mergeHadoopConf(job);
-		this.jobConf = job;
-	}
-
-	@Override
-	public void configure(Configuration parameters) {
-		
-	}
-
-	@Override
-	public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException {
-		return null;
-	}
-
-	@Override
-	public HadoopInputSplit[] createInputSplits(int minNumSplits)
-			throws IOException {
-		org.apache.hadoop.mapred.InputSplit[] splitArray = hadoopInputFormat.getSplits(jobConf, minNumSplits);
-		HadoopInputSplit[] hiSplit = new HadoopInputSplit[splitArray.length];
-		for(int i=0;i<splitArray.length;i++){
-			hiSplit[i] = new HadoopInputSplit(i, splitArray[i], jobConf);
-		}
-		return hiSplit;
-	}
-
-	@Override
-	public InputSplitAssigner getInputSplitAssigner(HadoopInputSplit[] inputSplits) {
-		return new DefaultInputSplitAssigner(inputSplits);
-	}
-
-	@Override
-	public void open(HadoopInputSplit split) throws IOException {
-		this.recordReader = this.hadoopInputFormat.getRecordReader(split.getHadoopInputSplit(), jobConf, new HadoopDummyReporter());
-		key = this.recordReader.createKey();
-		value = this.recordReader.createValue();
-		this.fetched = false;
-	}
-
-	private void fetchNext() throws IOException {
-		hasNext = this.recordReader.next(key, value);
-		fetched = true;
-	}
-	
-	@Override
-	public boolean reachedEnd() throws IOException {
-		if(!fetched) {
-			fetchNext();
-		}
-		return !hasNext;
-	}
-
-	@Override
-	public Record nextRecord(Record record) throws IOException {
-		if(!fetched) {
-			fetchNext();
-		}
-		if(!hasNext) {
-			return null;
-		}
-		converter.convert(record, key, value);
-		fetched = false;
-		return record;
-	}
-
-	@Override
-	public void close() throws IOException {
-		this.recordReader.close();
-	}
-	
-	/**
-	 * Custom serialization methods.
-	 *  @see "http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html"
-	 */
-	private void writeObject(ObjectOutputStream out) throws IOException {
-		out.writeUTF(hadoopInputFormatName);
-		jobConf.write(out);
-		out.writeObject(converter);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		hadoopInputFormatName = in.readUTF();
-		if(jobConf == null) {
-			jobConf = new JobConf();
-		}
-		jobConf.readFields(in);
-		try {
-			this.hadoopInputFormat = (org.apache.hadoop.mapred.InputFormat<K,V>) Class.forName(this.hadoopInputFormatName).newInstance();
-		} catch (Exception e) {
-			throw new RuntimeException("Unable to instantiate the hadoop input format", e);
-		}
-		ReflectionUtils.setConf(hadoopInputFormat, jobConf);
-		converter = (HadoopTypeConverter<K,V>) in.readObject();
-	}
-	
-	public void setJobConf(JobConf job) {
-		this.jobConf = job;
-	}
-		
-
-	public org.apache.hadoop.mapred.InputFormat<K,V> getHadoopInputFormat() {
-		return hadoopInputFormat;
-	}
-	
-	public void setHadoopInputFormat(org.apache.hadoop.mapred.InputFormat<K,V> hadoopInputFormat) {
-		this.hadoopInputFormat = hadoopInputFormat;
-	}
-	
-	public JobConf getJobConf() {
-		return jobConf;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
deleted file mode 100644
index e519062..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.HadoopFileOutputCommitter;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.FlinkTypeConverter;
-import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyProgressable;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
-import org.apache.flink.types.Record;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.util.ReflectionUtils;
-
-
-public class HadoopRecordOutputFormat<K,V> implements OutputFormat<Record> {
-
-	private static final long serialVersionUID = 1L;
-
-	public JobConf jobConf;
-
-	public org.apache.hadoop.mapred.OutputFormat<K,V> hadoopOutputFormat;
-
-	private String hadoopOutputFormatName;
-
-	public RecordWriter<K,V> recordWriter;
-
-	public FlinkTypeConverter<K,V> converter;
-
-	public HadoopFileOutputCommitter fileOutputCommitterWrapper;
-
-	public HadoopRecordOutputFormat(org.apache.hadoop.mapred.OutputFormat<K,V> hadoopFormat, JobConf job, FlinkTypeConverter<K,V> conv) {
-		super();
-		this.hadoopOutputFormat = hadoopFormat;
-		this.hadoopOutputFormatName = hadoopFormat.getClass().getName();
-		this.converter = conv;
-		this.fileOutputCommitterWrapper = new HadoopFileOutputCommitter();
-		HadoopUtils.mergeHadoopConf(job);
-		this.jobConf = job;
-	}
-
-	@Override
-	public void configure(Configuration parameters) {
-	}
-
-	/**
-	 * create the temporary output file for hadoop RecordWriter.
-	 * @param taskNumber The number of the parallel instance.
-	 * @param numTasks The number of parallel tasks.
-	 * @throws IOException
-	 */
-	@Override
-	public void open(int taskNumber, int numTasks) throws IOException {
-		this.fileOutputCommitterWrapper.setupJob(this.jobConf);
-		if (Integer.toString(taskNumber + 1).length() <= 6) {
-			this.jobConf.set("mapred.task.id", "attempt__0000_r_" + String.format("%" + (6 - Integer.toString(taskNumber + 1).length()) + "s"," ").replace(" ", "0") + Integer.toString(taskNumber + 1) + "_0");
-			//compatible for hadoop 2.2.0, the temporary output directory is different from hadoop 1.2.1
-			this.jobConf.set("mapreduce.task.output.dir", this.fileOutputCommitterWrapper.getTempTaskOutputPath(this.jobConf,TaskAttemptID.forName(this.jobConf.get("mapred.task.id"))).toString());
-		} else {
-			throw new IOException("task id too large");
-		}
-		this.recordWriter = this.hadoopOutputFormat.getRecordWriter(null, this.jobConf, Integer.toString(taskNumber + 1), new HadoopDummyProgressable());
-	}
-
-
-	@Override
-	public void writeRecord(Record record) throws IOException {
-		K key = this.converter.convertKey(record);
-		V value = this.converter.convertValue(record);
-		this.recordWriter.write(key, value);
-	}
-
-	/**
-	 * commit the task by moving the output file out from the temporary directory.
-	 * @throws IOException
-	 */
-	@Override
-	public void close() throws IOException {
-		this.recordWriter.close(new HadoopDummyReporter());
-		if (this.fileOutputCommitterWrapper.needsTaskCommit(this.jobConf, TaskAttemptID.forName(this.jobConf.get("mapred.task.id")))) {
-			this.fileOutputCommitterWrapper.commitTask(this.jobConf, TaskAttemptID.forName(this.jobConf.get("mapred.task.id")));
-		}
-	//TODO: commitjob when all the tasks are finished
-	}
-
-
-	/**
-	 * Custom serialization methods.
-	 *  @see "http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html"
-	 */
-	private void writeObject(ObjectOutputStream out) throws IOException {
-		out.writeUTF(hadoopOutputFormatName);
-		jobConf.write(out);
-		out.writeObject(converter);
-		out.writeObject(fileOutputCommitterWrapper);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		hadoopOutputFormatName = in.readUTF();
-		if(jobConf == null) {
-			jobConf = new JobConf();
-		}
-		jobConf.readFields(in);
-		try {
-			this.hadoopOutputFormat = (org.apache.hadoop.mapred.OutputFormat<K,V>) Class.forName(this.hadoopOutputFormatName).newInstance();
-		} catch (Exception e) {
-			throw new RuntimeException("Unable to instantiate the hadoop output format", e);
-		}
-		ReflectionUtils.setConf(hadoopOutputFormat, jobConf);
-		converter = (FlinkTypeConverter<K,V>) in.readObject();
-		fileOutputCommitterWrapper = (HadoopFileOutputCommitter) in.readObject();
-	}
-
-
-	public void setJobConf(JobConf job) {
-		this.jobConf = job;
-	}
-
-	public JobConf getJobConf() {
-		return jobConf;
-	}
-
-	public org.apache.hadoop.mapred.OutputFormat<K,V> getHadoopOutputFormat() {
-		return hadoopOutputFormat;
-	}
-
-	public void setHadoopOutputFormat(org.apache.hadoop.mapred.OutputFormat<K,V> hadoopOutputFormat) {
-		this.hadoopOutputFormat = hadoopOutputFormat;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultFlinkTypeConverter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultFlinkTypeConverter.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultFlinkTypeConverter.java
deleted file mode 100644
index 9d37988..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultFlinkTypeConverter.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import org.apache.flink.types.BooleanValue;
-import org.apache.flink.types.ByteValue;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.FloatValue;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.types.StringValue;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.ByteWritable;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-
-/**
- * Convert Flink Record into the default hadoop writables.
- */
-public class DefaultFlinkTypeConverter<K,V> implements FlinkTypeConverter<K,V> {
-	private static final long serialVersionUID = 1L;
-
-	private Class<K> keyClass;
-	private Class<V> valueClass;
-
-	public DefaultFlinkTypeConverter(Class<K> keyClass, Class<V> valueClass) {
-		this.keyClass= keyClass;
-		this.valueClass = valueClass;
-	}
-	@Override
-	public K convertKey(Record flinkRecord) {
-		if(flinkRecord.getNumFields() > 0) {
-			return convert(flinkRecord, 0, this.keyClass);
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public V convertValue(Record flinkRecord) {
-		if(flinkRecord.getNumFields() > 1) {
-			return convert(flinkRecord, 1, this.valueClass);
-		} else {
-			return null;
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	private<T> T convert(Record flinkType, int pos, Class<T> hadoopType) {
-		if(hadoopType == LongWritable.class ) {
-			return (T) new LongWritable((flinkType.getField(pos, LongValue.class)).getValue());
-		}
-		if(hadoopType == org.apache.hadoop.io.Text.class) {
-			return (T) new Text((flinkType.getField(pos, StringValue.class)).getValue());
-		}
-		if(hadoopType == org.apache.hadoop.io.IntWritable.class) {
-			return (T) new IntWritable((flinkType.getField(pos, IntValue.class)).getValue());
-		}
-		if(hadoopType == org.apache.hadoop.io.FloatWritable.class) {
-			return (T) new FloatWritable((flinkType.getField(pos, FloatValue.class)).getValue());
-		}
-		if(hadoopType == org.apache.hadoop.io.DoubleWritable.class) {
-			return (T) new DoubleWritable((flinkType.getField(pos, DoubleValue.class)).getValue());
-		}
-		if(hadoopType == org.apache.hadoop.io.BooleanWritable.class) {
-			return (T) new BooleanWritable((flinkType.getField(pos, BooleanValue.class)).getValue());
-		}
-		if(hadoopType == org.apache.hadoop.io.ByteWritable.class) {
-			return (T) new ByteWritable((flinkType.getField(pos, ByteValue.class)).getValue());
-		}
-
-		throw new RuntimeException("Unable to convert Flink type ("+flinkType.getClass().getCanonicalName()+") to Hadoop.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultHadoopTypeConverter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultHadoopTypeConverter.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultHadoopTypeConverter.java
deleted file mode 100644
index 6ed670a..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/DefaultHadoopTypeConverter.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import org.apache.flink.types.BooleanValue;
-import org.apache.flink.types.ByteValue;
-import org.apache.flink.types.DoubleValue;
-import org.apache.flink.types.FloatValue;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.types.StringValue;
-import org.apache.flink.types.Value;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.ByteWritable;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-
-
-/**
- * Converter for the default hadoop writables.
- * Key will be in field 0, Value in field 1 of a Record.
- */
-public class DefaultHadoopTypeConverter<K, V> implements HadoopTypeConverter<K, V> {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public void convert(Record flinkRecord, K hadoopKey, V hadoopValue) {
-		flinkRecord.setField(0, convert(hadoopKey));
-		flinkRecord.setField(1, convert(hadoopValue));
-	}
-	
-	protected Value convert(Object hadoopType) {
-		if(hadoopType instanceof org.apache.hadoop.io.LongWritable ) {
-			return new LongValue(((LongWritable)hadoopType).get());
-		}
-		if(hadoopType instanceof org.apache.hadoop.io.Text) {
-			return new StringValue(((Text)hadoopType).toString());
-		}
-		if(hadoopType instanceof org.apache.hadoop.io.IntWritable) {
-			return new IntValue(((IntWritable)hadoopType).get());
-		}
-		if(hadoopType instanceof org.apache.hadoop.io.FloatWritable) {
-			return new FloatValue(((FloatWritable)hadoopType).get());
-		}
-		if(hadoopType instanceof org.apache.hadoop.io.DoubleWritable) {
-			return new DoubleValue(((DoubleWritable)hadoopType).get());
-		}
-		if(hadoopType instanceof org.apache.hadoop.io.BooleanWritable) {
-			return new BooleanValue(((BooleanWritable)hadoopType).get());
-		}
-		if(hadoopType instanceof org.apache.hadoop.io.ByteWritable) {
-			return new ByteValue(((ByteWritable)hadoopType).get());
-		}
-		if (hadoopType instanceof NullWritable) {
-			return NullValue.getInstance();
-		}
-		
-		throw new RuntimeException("Unable to convert Hadoop type ("+hadoopType.getClass().getCanonicalName()+") to a Flink data type.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/FlinkTypeConverter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/FlinkTypeConverter.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/FlinkTypeConverter.java
deleted file mode 100644
index 3c14a86..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/FlinkTypeConverter.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import java.io.Serializable;
-
-import org.apache.flink.types.Record;
-
-/**
- * An interface describing a class that is able to
- * convert Flink's Record into Hadoop types model.
- *
- * The converter must be Serializable.
- *
- * Flink provides a DefaultFlinkTypeConverter. Custom implementations should
- * chain the type converters.
- */
-public interface FlinkTypeConverter<K,V> extends Serializable {
-
-	/**
-	 * Convert a Flink type to a Hadoop type.
-	 */
-	public K convertKey(Record record);
-
-	public V convertValue(Record record);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
deleted file mode 100644
index ce4955c..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopFileOutputCommitter.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.URI;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- * Hadoop 1.2.1 {@link org.apache.hadoop.mapred.FileOutputCommitter} takes {@link org.apache.hadoop.mapred.JobContext}
- * as input parameter. However JobContext class is package private, and in Hadoop 2.2.0 it's public.
- * This class takes {@link org.apache.hadoop.mapred.JobConf} as input instead of JobContext in order to setup and commit tasks.
- */
-public class HadoopFileOutputCommitter extends FileOutputCommitter implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	
-	static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER =
-		"mapreduce.fileoutputcommitter.marksuccessfuljobs";
-
-	public void setupJob(JobConf conf) throws IOException {
-		Path outputPath = FileOutputFormat.getOutputPath(conf);
-		if (outputPath != null) {
-			Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME);
-			FileSystem fileSys = tmpDir.getFileSystem(conf);
-			if (!fileSys.mkdirs(tmpDir)) {
-				LOG.error("Mkdirs failed to create " + tmpDir.toString());
-			}
-		}
-	}
-
-	private static boolean getOutputDirMarking(JobConf conf) {
-		return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,  true);
-	}
-
-	private void markSuccessfulOutputDir(JobConf conf)
-		throws IOException {
-		Path outputPath = FileOutputFormat.getOutputPath(conf);
-		if (outputPath != null) {
-			FileSystem fileSys = outputPath.getFileSystem(conf);
-			// create a file in the folder to mark it
-			if (fileSys.exists(outputPath)) {
-				Path filePath = new Path(outputPath, SUCCEEDED_FILE_NAME);
-				fileSys.create(filePath).close();
-			}
-		}
-	}
-
-	private Path getFinalPath(Path jobOutputDir, Path taskOutput,
-							Path taskOutputPath) throws IOException {
-		URI taskOutputUri = taskOutput.toUri();
-		URI relativePath = taskOutputPath.toUri().relativize(taskOutputUri);
-		if (taskOutputUri == relativePath) {//taskOutputPath is not a parent of taskOutput
-			throw new IOException("Can not get the relative path: base = " +
-				taskOutputPath + " child = " + taskOutput);
-		}
-		if (relativePath.getPath().length() > 0) {
-			return new Path(jobOutputDir, relativePath.getPath());
-		} else {
-			return jobOutputDir;
-		}
-	}
-	private void moveTaskOutputs(JobConf conf, TaskAttemptID taskAttemptID,
-								FileSystem fs,
-								Path jobOutputDir,
-								Path taskOutput)
-		throws IOException {
-		if (fs.isFile(taskOutput)) {
-			Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput,
-				getTempTaskOutputPath(conf, taskAttemptID));
-			if (!fs.rename(taskOutput, finalOutputPath)) {
-				if (!fs.delete(finalOutputPath, true)) {
-					throw new IOException("Failed to delete earlier output of task: " +
-						taskAttemptID);
-				}
-				if (!fs.rename(taskOutput, finalOutputPath)) {
-					throw new IOException("Failed to save output of task: " +
-						taskAttemptID);
-				}
-			}
-			LOG.debug("Moved " + taskOutput + " to " + finalOutputPath);
-		} else if(fs.getFileStatus(taskOutput).isDir()) {
-			FileStatus[] paths = fs.listStatus(taskOutput);
-			Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput,
-				getTempTaskOutputPath(conf, taskAttemptID));
-			fs.mkdirs(finalOutputPath);
-			if (paths != null) {
-				for (FileStatus path : paths) {
-					moveTaskOutputs(conf,taskAttemptID, fs, jobOutputDir, path.getPath());
-				}
-			}
-		}
-	}
-
-	public void commitTask(JobConf conf, TaskAttemptID taskAttemptID)
-		throws IOException {
-		Path taskOutputPath = getTempTaskOutputPath(conf, taskAttemptID);
-		if (taskOutputPath != null) {
-			FileSystem fs = taskOutputPath.getFileSystem(conf);
-			if (fs.exists(taskOutputPath)) {
-				Path jobOutputPath = taskOutputPath.getParent().getParent();
-				// Move the task outputs to their final place
-				moveTaskOutputs(conf,taskAttemptID, fs, jobOutputPath, taskOutputPath);
-				// Delete the temporary task-specific output directory
-				if (!fs.delete(taskOutputPath, true)) {
-					LOG.info("Failed to delete the temporary output" +
-						" directory of task: " + taskAttemptID + " - " + taskOutputPath);
-				}
-				LOG.info("Saved output of task '" + taskAttemptID + "' to " +
-					jobOutputPath);
-			}
-		}
-	}
-	public boolean needsTaskCommit(JobConf conf, TaskAttemptID taskAttemptID)
-		throws IOException {
-		try {
-			Path taskOutputPath = getTempTaskOutputPath(conf, taskAttemptID);
-			if (taskOutputPath != null) {
-				// Get the file-system for the task output directory
-				FileSystem fs = taskOutputPath.getFileSystem(conf);
-				// since task output path is created on demand,
-				// if it exists, task needs a commit
-				if (fs.exists(taskOutputPath)) {
-					return true;
-				}
-			}
-		} catch (IOException  ioe) {
-			throw ioe;
-		}
-		return false;
-	}
-
-	public Path getTempTaskOutputPath(JobConf conf, TaskAttemptID taskAttemptID) {
-		Path outputPath = FileOutputFormat.getOutputPath(conf);
-		if (outputPath != null) {
-			Path p = new Path(outputPath,
-				(FileOutputCommitter.TEMP_DIR_NAME + Path.SEPARATOR +
-					"_" + taskAttemptID.toString()));
-			try {
-				FileSystem fs = p.getFileSystem(conf);
-				return p.makeQualified(fs);
-			} catch (IOException ie) {
-				LOG.warn(StringUtils.stringifyException(ie));
-				return p;
-			}
-		}
-		return null;
-	}
-	public void cleanupJob(JobConf conf) throws IOException {
-		// do the clean up of temporary directory
-		Path outputPath = FileOutputFormat.getOutputPath(conf);
-		if (outputPath != null) {
-			Path tmpDir = new Path(outputPath, FileOutputCommitter.TEMP_DIR_NAME);
-			FileSystem fileSys = tmpDir.getFileSystem(conf);
-			if (fileSys.exists(tmpDir)) {
-				fileSys.delete(tmpDir, true);
-			}
-		} else {
-			LOG.warn("Output path is null in cleanup");
-		}
-	}
-
-	public void commitJob(JobConf conf) throws IOException {
-		cleanupJob(conf);
-		if (getOutputDirMarking(conf)) {
-			markSuccessfulOutputDir(conf);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopTypeConverter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopTypeConverter.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopTypeConverter.java
deleted file mode 100644
index 6bbf077..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/HadoopTypeConverter.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import java.io.Serializable;
-
-import org.apache.flink.types.Record;
-
-
-/**
- * An interface describing a class that is able to 
- * convert Hadoop types into Flink's Record model.
- * 
- * The converter must be Serializable.
- * 
- * Flink provides a DefaultHadoopTypeConverter. Custom implementations should
- * chain the type converters.
- */
-public interface HadoopTypeConverter<K, V> extends Serializable {
-	
-	/**
-	 * Convert a Hadoop type to a Flink type.
-	 */
-	public void convert(Record record, K hadoopKey, V hadoopValue);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableComparableWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableComparableWrapper.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableComparableWrapper.java
deleted file mode 100644
index 0519ac9..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableComparableWrapper.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import org.apache.flink.types.Key;
-import org.apache.hadoop.io.WritableComparable;
-
-public class WritableComparableWrapper<T extends WritableComparable<T>> extends WritableWrapper<T> implements Key<WritableComparableWrapper<T>> {
-	private static final long serialVersionUID = 1L;
-	
-	public WritableComparableWrapper() {
-		super();
-	}
-	
-	public WritableComparableWrapper(T toWrap) {
-		super(toWrap);
-	}
-
-	@Override
-	public int compareTo(WritableComparableWrapper<T> o) {
-		return super.value().compareTo(o.value());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java
deleted file mode 100644
index 6369bb7..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.hadoop.io.Writable;
-
-public class WritableWrapper<T extends Writable> implements Value {
-	private static final long serialVersionUID = 2L;
-	
-	private T wrapped;
-	private String wrappedType;
-	private ClassLoader cl;
-	
-	public WritableWrapper() {
-	}
-	
-	public WritableWrapper(T toWrap) {
-		wrapped = toWrap;
-		wrappedType = toWrap.getClass().getCanonicalName();
-	}
-
-	public T value() {
-		return wrapped;
-	}
-	
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		out.writeUTF(wrappedType);
-		wrapped.write(out);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		if(cl == null) {
-			cl = Thread.currentThread().getContextClassLoader();
-		}
-		wrappedType = in.readUTF();
-		try {
-			@SuppressWarnings("unchecked")
-			Class<T> wrClass = (Class<T>) Class.forName(wrappedType, true, cl).asSubclass(Writable.class);
-			wrapped = InstantiationUtil.instantiate(wrClass, Writable.class);
-		} catch (ClassNotFoundException e) {
-			throw new RuntimeException("Error creating the WritableWrapper", e);
-		}
-		wrapped.readFields(in);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapperConverter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapperConverter.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapperConverter.java
deleted file mode 100644
index 6bb13d1..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/datatypes/WritableWrapperConverter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.hadoopcompatibility.mapred.record.datatypes;
-
-import org.apache.flink.types.Record;
-import org.apache.flink.types.Value;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-@SuppressWarnings("rawtypes")
-public class WritableWrapperConverter<K extends WritableComparable, V extends Writable> implements HadoopTypeConverter<K,V> {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public void convert(Record flinkRecord, K hadoopKey, V hadoopValue) {
-		flinkRecord.setField(0, convertKey(hadoopKey));
-		flinkRecord.setField(1, convertValue(hadoopValue));
-	}
-	
-	@SuppressWarnings("unchecked")
-	private final Value convertKey(K in) {
-		return new WritableComparableWrapper(in);
-	}
-	
-	private final Value convertValue(V in) {
-		return new WritableWrapper<V>(in);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java
deleted file mode 100644
index b167080..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCount.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred.record.example;
-
-import java.io.Serializable;
-import java.util.Iterator;
-import java.util.StringTokenizer;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.Program;
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.java.record.functions.MapFunction;
-import org.apache.flink.api.java.record.functions.ReduceFunction;
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields;
-import org.apache.flink.api.java.record.io.CsvOutputFormat;
-import org.apache.flink.api.java.record.operators.FileDataSink;
-import org.apache.flink.api.java.record.operators.MapOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable;
-import org.apache.flink.client.LocalExecutor;
-import org.apache.flink.hadoopcompatibility.mapred.record.HadoopDataSource;
-import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.WritableWrapperConverter;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.types.StringValue;
-import org.apache.flink.util.Collector;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TextInputFormat;
-
-/**
- * Implements a word count which takes the input file and counts the number of
- * the occurrences of each word in the file. 
- * 
- * <br /><br />
- * 
- * <b>Note</b>: This example uses the out-dated Record API.
- * It is recommended to use the new Java API.
- * 
- * @see org.apache.flink.hadoopcompatibility.mapred.example.HadoopMapredCompatWordCount
- */
-public class WordCount implements Program, ProgramDescription {
-	
-	private static final long serialVersionUID = 1L;
-
-
-	/**
-	 * Converts a Record containing one string in to multiple string/integer pairs.
-	 * The string is tokenized by whitespaces. For each token a new record is emitted,
-	 * where the token is the first field and an Integer(1) is the second field.
-	 */
-	public static class TokenizeLine extends MapFunction implements Serializable {
-		private static final long serialVersionUID = 1L;
-		
-		@Override
-		public void map(Record record, Collector<Record> collector) {
-			// get the first field (as type StringValue) from the record
-			String line = record.getField(1, StringValue.class).getValue();
-			// normalize the line
-			line = line.replaceAll("\\W+", " ").toLowerCase();
-			
-			// tokenize the line
-			StringTokenizer tokenizer = new StringTokenizer(line);
-			while (tokenizer.hasMoreTokens()) {
-				String word = tokenizer.nextToken();
-				
-				// we emit a (word, 1) pair 
-				collector.collect(new Record(new StringValue(word), new IntValue(1)));
-			}
-		}
-	}
-
-	/**
-	 * Sums up the counts for a certain given key. The counts are assumed to be at position <code>1</code>
-	 * in the record. The other fields are not modified.
-	 */
-	@Combinable
-	@ConstantFields(0)
-	public static class CountWords extends ReduceFunction implements Serializable {
-		
-		private static final long serialVersionUID = 1L;
-		
-		@Override
-		public void reduce(Iterator<Record> records, Collector<Record> out) throws Exception {
-			Record element = null;
-			int sum = 0;
-			
-			while (records.hasNext()) {
-				element = records.next();
-				int cnt = element.getField(1, IntValue.class).getValue();
-				sum += cnt;
-			}
-
-			element.setField(1, new IntValue(sum));
-			out.collect(element);
-		}
-		
-		@Override
-		public void combine(Iterator<Record> records, Collector<Record> out) throws Exception {
-			// the logic is the same as in the reduce function, so simply call the reduce method
-			reduce(records, out);
-		}
-	}
-
-
-	@SuppressWarnings({ "rawtypes", "unchecked", "unused" })
-	@Override
-	public Plan getPlan(String... args) {
-		// parse job parameters
-		int numSubTasks   = (args.length > 0 ? Integer.parseInt(args[0]) : 1);
-		String dataInput = (args.length > 1 ? args[1] : "");
-		String output    = (args.length > 2 ? args[2] : "");
-		
-		
-		HadoopDataSource source = new HadoopDataSource(new TextInputFormat(), new JobConf(), "Input Lines");
-		TextInputFormat.addInputPath(source.getJobConf(), new Path(dataInput));
-		
-		// Example with Wrapper Converter
-		HadoopDataSource<LongWritable,Text> sourceHadoopType = new HadoopDataSource<LongWritable, Text>(
-				new TextInputFormat(), new JobConf(), "Input Lines", new WritableWrapperConverter<LongWritable, Text>());
-		TextInputFormat.addInputPath(source.getJobConf(), new Path(dataInput));
-		
-		MapOperator mapper = MapOperator.builder(new TokenizeLine())
-			.input(source)
-			.name("Tokenize Lines")
-			.build();
-		ReduceOperator reducer = ReduceOperator.builder(CountWords.class, StringValue.class, 0)
-			.input(mapper)
-			.name("Count Words")
-			.build();
-		FileDataSink out = new FileDataSink(new CsvOutputFormat(), output, reducer, "Word Counts");
-		CsvOutputFormat.configureRecordFormat(out)
-			.recordDelimiter('\n')
-			.fieldDelimiter(' ')
-			.field(StringValue.class, 0)
-			.field(IntValue.class, 1);
-		
-		Plan plan = new Plan(out, "WordCount Example");
-		plan.setDefaultParallelism(numSubTasks);
-		return plan;
-	}
-
-
-	@Override
-	public String getDescription() {
-		return "Parameters: [numSubStasks] [input] [output]";
-	}
-
-	
-	public static void main(String[] args) throws Exception {
-		WordCount wc = new WordCount();
-		
-		if (args.length < 3) {
-			System.err.println(wc.getDescription());
-			System.exit(1);
-		}
-		
-		Plan plan = wc.getPlan(args);
-		
-		// This will execute the word-count embedded in a local context. replace this line by the commented
-		// succeeding line to send the job to a local installation or to a cluster for execution
-		LocalExecutor.execute(plan);
-//		PlanExecutor ex = new RemoteExecutor("localhost", 6123, "target/pact-examples-0.4-SNAPSHOT-WordCount.jar");
-//		ex.executePlan(plan);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java
deleted file mode 100644
index a838215..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/example/WordCountWithOutputFormat.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.flink.hadoopcompatibility.mapred.record.example;
-
-import java.io.Serializable;
-import java.util.Iterator;
-import java.util.StringTokenizer;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.Program;
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.java.record.functions.MapFunction;
-import org.apache.flink.api.java.record.functions.ReduceFunction;
-import org.apache.flink.api.java.record.functions.FunctionAnnotation.ConstantFields;
-import org.apache.flink.api.java.record.operators.MapOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator;
-import org.apache.flink.api.java.record.operators.ReduceOperator.Combinable;
-import org.apache.flink.client.LocalExecutor;
-import org.apache.flink.hadoopcompatibility.mapred.record.HadoopDataSink;
-import org.apache.flink.hadoopcompatibility.mapred.record.HadoopDataSource;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.Record;
-import org.apache.flink.types.StringValue;
-import org.apache.flink.util.Collector;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.TextOutputFormat;
-
-/**
- * Implements a word count which takes the input file and counts the number of
- * the occurrences of each word in the file.
- * 
- * <br /><br />
- * 
- * <b>Note</b>: This example uses the out dated Record API.
- * It is recommended to use the new Java API.
- * 
- * @see WordCount
- */
-@SuppressWarnings("serial")
-public class WordCountWithOutputFormat implements Program, ProgramDescription {
-
-	/**
-	 * Converts a Record containing one string in to multiple string/integer pairs.
-	 * The string is tokenized by whitespaces. For each token a new record is emitted,
-	 * where the token is the first field and an Integer(1) is the second field.
-	 */
-	public static class TokenizeLine extends MapFunction implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void map(Record record, Collector<Record> collector) {
-			// get the first field (as type StringValue) from the record
-			String line = record.getField(1, StringValue.class).getValue();
-			// normalize the line
-			line = line.replaceAll("\\W+", " ").toLowerCase();
-
-			// tokenize the line
-			StringTokenizer tokenizer = new StringTokenizer(line);
-			while (tokenizer.hasMoreTokens()) {
-				String word = tokenizer.nextToken();
-
-				// we emit a (word, 1) pair 
-				collector.collect(new Record(new StringValue(word), new IntValue(1)));
-			}
-		}
-	}
-
-	/**
-	 * Sums up the counts for a certain given key. The counts are assumed to be at position <code>1</code>
-	 * in the record. The other fields are not modified.
-	 */
-	@Combinable
-	@ConstantFields(0)
-	public static class CountWords extends ReduceFunction implements Serializable {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void reduce(Iterator<Record> records, Collector<Record> out) throws Exception {
-			Record element = null;
-			int sum = 0;
-			
-			while (records.hasNext()) {
-				element = records.next();
-				int cnt = element.getField(1, IntValue.class).getValue();
-				sum += cnt;
-			}
-
-			element.setField(1, new IntValue(sum));
-			out.collect(element);
-		}
-
-		@Override
-		public void combine(Iterator<Record> records, Collector<Record> out) throws Exception {
-			// the logic is the same as in the reduce function, so simply call the reduce method
-			reduce(records, out);
-		}
-	}
-
-
-	@Override
-	public Plan getPlan(String... args) {
-		// parse job parameters
-		int numSubTasks   = (args.length > 0 ? Integer.parseInt(args[0]) : 1);
-		String dataInput = (args.length > 1 ? args[1] : "");
-		String output    = (args.length > 2 ? args[2] : "");
-
-		HadoopDataSource<LongWritable, Text> source = new HadoopDataSource<LongWritable, Text>(
-				new TextInputFormat(), new JobConf(), "Input Lines");
-		TextInputFormat.addInputPath(source.getJobConf(), new Path(dataInput));
-
-
-		MapOperator mapper = MapOperator.builder(new TokenizeLine())
-				.input(source)
-				.name("Tokenize Lines")
-				.build();
-		ReduceOperator reducer = ReduceOperator.builder(CountWords.class, StringValue.class, 0)
-				.input(mapper)
-				.name("Count Words")
-				.build();
-		HadoopDataSink<Text, IntWritable> out = new HadoopDataSink<Text, IntWritable>(new TextOutputFormat<Text, IntWritable>(),new JobConf(), "Hadoop TextOutputFormat", reducer, Text.class, IntWritable.class);
-		TextOutputFormat.setOutputPath(out.getJobConf(), new Path(output));
-
-		Plan plan = new Plan(out, "Hadoop OutputFormat Example");
-		plan.setDefaultParallelism(numSubTasks);
-		return plan;
-	}
-
-
-	@Override
-	public String getDescription() {
-		return "Parameters: [numSubStasks] [input] [output]";
-	}
-
-
-	public static void main(String[] args) throws Exception {
-		WordCountWithOutputFormat wc = new WordCountWithOutputFormat();
-
-		if (args.length < 3) {
-			System.err.println(wc.getDescription());
-			System.exit(1);
-		}
-
-		Plan plan = wc.getPlan(args);
-
-		// This will execute the word-count embedded in a local context. replace this line by the commented
-		// succeeding line to send the job to a local installation or to a cluster for execution
-		LocalExecutor.execute(plan);
-//		PlanExecutor ex = new RemoteExecutor("localhost", 6123, "target/pact-examples-0.4-SNAPSHOT-WordCount.jar");
-//		ex.executePlan(plan);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/2ff04baa/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/record/HadoopRecordInputOutputITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/record/HadoopRecordInputOutputITCase.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/record/HadoopRecordInputOutputITCase.java
deleted file mode 100644
index fe7ea8e..0000000
--- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/record/HadoopRecordInputOutputITCase.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.
- */
-
-
-package org.apache.flink.test.hadoopcompatibility.mapred.record;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.hadoopcompatibility.mapred.record.example.WordCountWithOutputFormat;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.RecordAPITestBase;
-
-/**
- * test the hadoop inputformat and outputformat
- */
-public class HadoopRecordInputOutputITCase extends RecordAPITestBase {
-	protected String textPath;
-	protected String resultPath;
-	protected String counts;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-		counts = WordCountData.COUNTS.replaceAll(" ", "\t");
-	}
-
-	@Override
-	protected Plan getTestJob() {
-		//WordCountWithHadoopOutputFormat takes hadoop TextInputFormat as input and output file in hadoop TextOutputFormat
-		WordCountWithOutputFormat wc = new WordCountWithOutputFormat();
-		return wc.getPlan("1", textPath, resultPath);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		// Test results, append /1 to resultPath due to the generated _temproray file.
-		compareResultsByLinesInMemory(counts, resultPath + "/1");
-	}
-}


[03/13] flink git commit: [FLINK-2920] [tests] Port KryoVersusAvroMinibenchmark to JMH.

Posted by fh...@apache.org.
[FLINK-2920] [tests] Port KryoVersusAvroMinibenchmark to JMH.

This closes #1302


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

Branch: refs/heads/master
Commit: a8f1be923a4c4954fb0ae71a9c0ee672aaf2d797
Parents: 3abbcd1
Author: gallenvara <ga...@126.com>
Authored: Tue Oct 27 11:27:03 2015 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 10:32:21 2015 +0100

----------------------------------------------------------------------
 flink-benchmark/pom.xml                         | 173 +++---
 .../kryo/KryoVersusAvroMinibenchmark.java       | 518 ++++++++++++++++++
 .../kryo/KryoVersusAvroMinibenchmark.java       | 522 -------------------
 3 files changed, 604 insertions(+), 609 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a8f1be92/flink-benchmark/pom.xml
----------------------------------------------------------------------
diff --git a/flink-benchmark/pom.xml b/flink-benchmark/pom.xml
index 6b2c13d..c60d522 100644
--- a/flink-benchmark/pom.xml
+++ b/flink-benchmark/pom.xml
@@ -16,99 +16,98 @@ software distributed under the License is distributed on an
 KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
-
 -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
-  <modelVersion>4.0.0</modelVersion>
+	<modelVersion>4.0.0</modelVersion>
 
-  <parent>
-    <groupId>org.apache.flink</groupId>
-    <artifactId>flink-parent</artifactId>
-    <version>1.0-SNAPSHOT</version>
-    <relativePath>..</relativePath>
-  </parent>
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>1.0-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
 
-  <artifactId>flink-benchmark</artifactId>
-  <name>flink-benchmark</name>
-  <packaging>jar</packaging>
+	<artifactId>flink-benchmark</artifactId>
+	<name>flink-benchmark</name>
+	<packaging>jar</packaging>
 
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <jmh.version>1.11</jmh.version>
-    <uberjar.name>benchmarks</uberjar.name>
-  </properties>
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+		<jmh.version>1.11</jmh.version>
+		<uberjar.name>benchmarks</uberjar.name>
+	</properties>
 
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-clients</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.openjdk.jmh</groupId>
-      <artifactId>jmh-core</artifactId>
-      <version>${jmh.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.openjdk.jmh</groupId>
-      <artifactId>jmh-generator-annprocess</artifactId>
-      <version>${jmh.version}</version>
-      <scope>provided</scope>
-    </dependency>  
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-runtime</artifactId>
-      <version>1.0-SNAPSHOT</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.flink</groupId>
-      <artifactId>flink-java</artifactId>
-      <version>${project.version}</version>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.openjdk.jmh</groupId>
+			<artifactId>jmh-core</artifactId>
+			<version>${jmh.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.openjdk.jmh</groupId>
+			<artifactId>jmh-generator-annprocess</artifactId>
+			<version>${jmh.version}</version>
+			<scope>provided</scope>
+		</dependency>	
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
 
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>shade</goal>
-            </goals>
-            <configuration>
-              <finalName>${uberjar.name}</finalName>
-              <transformers>
-                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                  <mainClass>org.openjdk.jmh.Main</mainClass>
-                </transformer>
-              </transformers>
-              <filters>
-                <filter>
-                  <!--
-                  Shading signed JARs will fail without this.
-                  http://stackoverflow.com/questions/999489/invalid-signature-file-when-attempting-to-run-a-jar
-                  -->
-                  <artifact>*:*</artifact>
-                  <excludes>
-                    <exclude>META-INF/*.SF</exclude>
-                    <exclude>META-INF/*.DSA</exclude>
-                    <exclude>META-INF/*.RSA</exclude>
-                  </excludes>
-                </filter>
-              </filters>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<finalName>${uberjar.name}</finalName>
+							<transformers>
+								<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+									<mainClass>org.openjdk.jmh.Main</mainClass>
+								</transformer>
+							</transformers>
+							<filters>
+								<filter>
+									<!--
+									Shading signed JARs will fail without this.
+									http://stackoverflow.com/questions/999489/invalid-signature-file-when-attempting-to-run-a-jar
+									-->
+									<artifact>*:*</artifact>
+									<excludes>
+										<exclude>META-INF/*.SF</exclude>
+										<exclude>META-INF/*.DSA</exclude>
+										<exclude>META-INF/*.RSA</exclude>
+									</excludes>
+								</filter>
+							</filters>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/a8f1be92/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java
new file mode 100644
index 0000000..98af91e
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java
@@ -0,0 +1,518 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.benchmark.api.java.typeutils.runtime.kryo;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.AvroSerializer;
+import org.apache.flink.api.java.typeutils.runtime.TestDataOutputSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.MemoryUtils;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+@State(Scope.Thread)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class KryoVersusAvroMinibenchmark {
+
+	private static final long SEED = 94762389741692387L;
+	
+	private static final Random rnd = new Random(SEED);
+	
+	private static final int NUM_ELEMENTS = 100000;
+
+	@Param({"1","2","3","4","5","6","7","8","9","10"})
+	private static int runTime;
+
+	private MyType[] elements ;
+
+	private MyType dummy;
+
+	@Setup
+	public void init() {
+		this.elements = new MyType[NUM_ELEMENTS];
+		for (int i = 0; i < NUM_ELEMENTS; i++) {
+			this.elements[i] = MyType.getRandom();
+		}
+		this.dummy = new MyType();
+	}
+
+	@Benchmark
+	public void avroSerializer() throws IOException {
+		final TestDataOutputSerializer outView = new TestDataOutputSerializer(10000000);
+		final AvroSerializer<MyType> serializer = new AvroSerializer<MyType>(MyType.class);
+
+		for (int k = 0; k < NUM_ELEMENTS; k++) {
+			serializer.serialize(elements[k], outView);
+		}
+
+		final DataInputDeserializer inView = new DataInputDeserializer(outView.wrapAsByteBuffer());
+		for (int k = 0; k < NUM_ELEMENTS; k++) {
+			serializer.deserialize(dummy, inView);
+		}
+	}
+
+	@Benchmark
+	public void kryoSerializer() throws IOException {
+		final TestDataOutputSerializer outView = new TestDataOutputSerializer(10000000);
+		ExecutionConfig conf = new ExecutionConfig();
+		conf.registerKryoType(MyType.class);
+		conf.enableForceKryo();
+		TypeInformation<MyType> typeInfo = new GenericTypeInfo<MyType>(MyType.class);
+		final TypeSerializer<MyType> serializer = typeInfo.createSerializer(conf);
+
+		for (int k = 0; k < NUM_ELEMENTS; k++) {
+			serializer.serialize(elements[k], outView);
+		}
+
+		final DataInputDeserializer inView = new DataInputDeserializer(outView.wrapAsByteBuffer());
+		for (int k = 0; k < NUM_ELEMENTS; k++) {
+			serializer.deserialize(dummy, inView);
+		}
+	}
+
+	public static void main(String[] args) throws Exception {
+
+		Options opt = new OptionsBuilder()
+				.include(KryoVersusAvroMinibenchmark.class.getSimpleName())
+				.warmupIterations(2)
+				.measurementIterations(2)
+				.forks(1)
+				.build();
+		new Runner(opt).run();
+	}
+
+	public static class MyType {
+		
+		private String theString;
+		
+//		private Tuple2<Long, Double> theTuple;
+		
+		private List<Integer> theList;
+
+		
+		public MyType() {
+			theString = "";
+//			theTuple = new Tuple2<Long, Double>(0L, 0.0);
+			theList = new ArrayList<Integer>();
+		}
+		
+		public MyType(String theString, Tuple2<Long, Double> theTuple, List<Integer> theList) {
+			this.theString = theString;
+//			this.theTuple = theTuple;
+			this.theList = theList;
+		}
+
+		
+		public String getTheString() {
+			return theString;
+		}
+
+		public void setTheString(String theString) {
+			this.theString = theString;
+		}
+
+//		public Tuple2<Long, Double> getTheTuple() {
+//			return theTuple;
+//		}
+//
+//		public void setTheTuple(Tuple2<Long, Double> theTuple) {
+//			this.theTuple = theTuple;
+//		}
+
+		public List<Integer> getTheList() {
+			return theList;
+		}
+
+		public void setTheList(List<Integer> theList) {
+			this.theList = theList;
+		}
+		
+		
+		public static MyType getRandom() {
+			final int numListElements = rnd.nextInt(20);
+			List<Integer> list = new ArrayList<Integer>(numListElements);
+			for (int i = 0; i < numListElements; i++) {
+				list.add(rnd.nextInt());
+			}
+			
+			return new MyType(randomString(), new Tuple2<Long, Double>(rnd.nextLong(), rnd.nextDouble()), list);
+		}
+	}
+	
+	
+	private static String randomString() {
+		final int len = rnd.nextInt(100) + 20;
+		
+		StringBuilder bld = new StringBuilder();
+		for (int i = 0; i < len; i++) {
+			bld.append(rnd.nextInt('z' - 'a' + 1) + 'a');
+		}
+		return bld.toString();
+	}
+	
+	// ============================================================================================
+	// ============================================================================================
+	
+	public static final class DataInputDeserializer implements DataInputView {
+		
+		private byte[] buffer;
+		
+		private int end;
+
+		private int position;
+
+		public DataInputDeserializer() {
+		}
+		
+		public DataInputDeserializer(byte[] buffer, int start, int len) {
+			setBuffer(buffer, start, len);
+		}
+		
+		public DataInputDeserializer(ByteBuffer buffer) {
+			setBuffer(buffer);
+		}
+
+		public void setBuffer(ByteBuffer buffer) {
+			if (buffer.hasArray()) {
+				this.buffer = buffer.array();
+				this.position = buffer.arrayOffset() + buffer.position();
+				this.end = this.position + buffer.remaining();
+			} else if (buffer.isDirect()) {
+				this.buffer = new byte[buffer.remaining()];
+				this.position = 0;
+				this.end = this.buffer.length;
+
+				buffer.get(this.buffer);
+			} else {
+				throw new IllegalArgumentException("The given buffer is neither an array-backed heap ByteBuffer, nor a direct ByteBuffer.");
+			}
+		}
+
+		public void setBuffer(byte[] buffer, int start, int len) {
+			if (buffer == null) {
+				throw new NullPointerException();
+			}
+
+			if (start < 0 || len < 0 || start + len >= buffer.length) {
+				throw new IllegalArgumentException();
+			}
+
+			this.buffer = buffer;
+			this.position = start;
+			this.end = start * len;
+		}
+
+		// ----------------------------------------------------------------------------------------
+		//                               Data Input
+		// ----------------------------------------------------------------------------------------
+		
+		@Override
+		public boolean readBoolean() throws IOException {
+			if (this.position < this.end) {
+				return this.buffer[this.position++] != 0;
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public byte readByte() throws IOException {
+			if (this.position < this.end) {
+				return this.buffer[this.position++];
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public char readChar() throws IOException {
+			if (this.position < this.end - 1) {
+				return (char) (((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0));
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public double readDouble() throws IOException {
+			return Double.longBitsToDouble(readLong());
+		}
+
+		@Override
+		public float readFloat() throws IOException {
+			return Float.intBitsToFloat(readInt());
+		}
+
+		@Override
+		public void readFully(byte[] b) throws IOException {
+			readFully(b, 0, b.length);
+		}
+
+		@Override
+		public void readFully(byte[] b, int off, int len) throws IOException {
+			if (len >= 0) {
+				if (off <= b.length - len) {
+					if (this.position <= this.end - len) {
+						System.arraycopy(this.buffer, position, b, off, len);
+						position += len;
+					} else {
+						throw new EOFException();
+					}
+				} else {
+					throw new ArrayIndexOutOfBoundsException();
+				}
+			} else if (len < 0) {
+				throw new IllegalArgumentException("Length may not be negative.");
+			}
+		}
+
+		@Override
+		public int readInt() throws IOException {
+			if (this.position >= 0 && this.position < this.end - 3) {
+				@SuppressWarnings("restriction")
+				int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
+				if (LITTLE_ENDIAN) {
+					value = Integer.reverseBytes(value);
+				}
+				
+				this.position += 4;
+				return value;
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public String readLine() throws IOException {
+			if (this.position < this.end) {
+				// read until a newline is found
+				StringBuilder bld = new StringBuilder();
+				char curr = (char) readUnsignedByte();
+				while (position < this.end && curr != '\n') {
+					bld.append(curr);
+					curr = (char) readUnsignedByte();
+				}
+				// trim a trailing carriage return
+				int len = bld.length();
+				if (len > 0 && bld.charAt(len - 1) == '\r') {
+					bld.setLength(len - 1);
+				}
+				String s = bld.toString();
+				bld.setLength(0);
+				return s;
+			} else {
+				return null;
+			}
+		}
+
+		@Override
+		public long readLong() throws IOException {
+			if (position >= 0 && position < this.end - 7) {
+				@SuppressWarnings("restriction")
+				long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
+				if (LITTLE_ENDIAN) {
+					value = Long.reverseBytes(value);
+				}
+				this.position += 8;
+				return value;
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public short readShort() throws IOException {
+			if (position >= 0 && position < this.end - 1) {
+				return (short) ((((this.buffer[position++]) & 0xff) << 8) | (((this.buffer[position++]) & 0xff) << 0));
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public String readUTF() throws IOException {
+			int utflen = readUnsignedShort();
+			byte[] bytearr = new byte[utflen];
+			char[] chararr = new char[utflen];
+
+			int c, char2, char3;
+			int count = 0;
+			int chararr_count = 0;
+
+			readFully(bytearr, 0, utflen);
+
+			while (count < utflen) {
+				c = (int) bytearr[count] & 0xff;
+				if (c > 127) {
+					break;
+				}
+				count++;
+				chararr[chararr_count++] = (char) c;
+			}
+
+			while (count < utflen) {
+				c = (int) bytearr[count] & 0xff;
+				switch (c >> 4) {
+				case 0:
+				case 1:
+				case 2:
+				case 3:
+				case 4:
+				case 5:
+				case 6:
+				case 7:
+					/* 0xxxxxxx */
+					count++;
+					chararr[chararr_count++] = (char) c;
+					break;
+				case 12:
+				case 13:
+					/* 110x xxxx 10xx xxxx */
+					count += 2;
+					if (count > utflen) {
+						throw new UTFDataFormatException("malformed input: partial character at end");
+					}
+					char2 = (int) bytearr[count - 1];
+					if ((char2 & 0xC0) != 0x80) {
+						throw new UTFDataFormatException("malformed input around byte " + count);
+					}
+					chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+					break;
+				case 14:
+					/* 1110 xxxx 10xx xxxx 10xx xxxx */
+					count += 3;
+					if (count > utflen) {
+						throw new UTFDataFormatException("malformed input: partial character at end");
+					}
+					char2 = (int) bytearr[count - 2];
+					char3 = (int) bytearr[count - 1];
+					if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
+						throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+					}
+					chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+					break;
+				default:
+					/* 10xx xxxx, 1111 xxxx */
+					throw new UTFDataFormatException("malformed input around byte " + count);
+				}
+			}
+			// The number of chars produced may be less than utflen
+			return new String(chararr, 0, chararr_count);
+		}
+
+		@Override
+		public int readUnsignedByte() throws IOException {
+			if (this.position < this.end) {
+				return (this.buffer[this.position++] & 0xff);
+			} else {
+				throw new EOFException();
+			}
+		}
+
+		@Override
+		public int readUnsignedShort() throws IOException {
+			if (this.position < this.end - 1) {
+				return ((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0);
+			} else {
+				throw new EOFException();
+			}
+		}
+		
+		@Override
+		public int skipBytes(int n) throws IOException {
+			if (this.position <= this.end - n) {
+				this.position += n;
+				return n;
+			} else {
+				n = this.end - this.position;
+				this.position = this.end;
+				return n;
+			}
+		}
+		
+		@SuppressWarnings("restriction")
+		private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+		
+		@SuppressWarnings("restriction")
+		private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+		
+		private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+
+		@Override
+		public void skipBytesToRead(int numBytes) throws IOException {
+			int skippedBytes = skipBytes(numBytes);
+
+			if (skippedBytes < numBytes){
+				throw new EOFException("Could not skip " + numBytes +" bytes.");
+			}
+		}
+
+		@Override
+		public int read(byte[] b, int off, int len) throws IOException {
+			if (b == null) {
+				throw new NullPointerException("Byte array b cannot be null.");
+			}
+
+			if (off < 0) {
+				throw new IndexOutOfBoundsException("Offset cannot be negative.");
+			}
+
+			if (len < 0) {
+				throw new IndexOutOfBoundsException("Length cannot be negative.");
+			}
+
+			if (b.length - off < len) {
+				throw new IndexOutOfBoundsException("Byte array does not provide enough space to store requested data" +
+						".");
+			}
+
+			if (this.position >= this.end) {
+				return -1;
+			} else {
+				int toRead = Math.min(this.end-this.position, len);
+				System.arraycopy(this.buffer,this.position,b,off,toRead);
+				this.position += toRead;
+
+				return toRead;
+			}
+		}
+
+		@Override
+		public int read(byte[] b) throws IOException {
+			return read(b, 0, b.length);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/a8f1be92/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java
deleted file mode 100644
index 99cfe79..0000000
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoVersusAvroMinibenchmark.java
+++ /dev/null
@@ -1,522 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.flink.api.java.typeutils.runtime.kryo;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.runtime.AvroSerializer;
-import org.apache.flink.api.java.typeutils.runtime.TestDataOutputSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.MemoryUtils;
-
-public class KryoVersusAvroMinibenchmark {
-
-	private static final long SEED = 94762389741692387L;
-	
-	private static final Random rnd = new Random(SEED);
-	
-	private static final int NUM_ELEMENTS = 100000;
-	
-	private static final int NUM_RUNS = 10;
-	
-	
-	
-	public static void main(String[] args) throws Exception {
-		
-		final MyType[] elements = new MyType[NUM_ELEMENTS];
-		for (int i = 0; i < NUM_ELEMENTS; i++) {
-			elements[i] = MyType.getRandom();
-		}
-		
-		final MyType dummy = new MyType();
-		
-		long[] timesAvro = new long[NUM_RUNS];
-		long[] timesKryo = new long[NUM_RUNS];
-		
-		for (int i = 0; i < NUM_RUNS; i++) {
-			System.out.println("----------------- Starting run " + i + " ---------------------");
-			
-			System.out.println("Avro serializer");
-			{
-				final TestDataOutputSerializer outView = new TestDataOutputSerializer(100000000);
-				final AvroSerializer<MyType> serializer = new AvroSerializer<MyType>(MyType.class);
-				
-				long start = System.nanoTime();
-				
-				for (int k = 0; k < NUM_ELEMENTS; k++) {
-					serializer.serialize(elements[k], outView);
-				}
-				
-				final DataInputDeserializer inView = new DataInputDeserializer(outView.wrapAsByteBuffer());
-				for (int k = 0; k < NUM_ELEMENTS; k++) {
-					serializer.deserialize(dummy, inView);
-				}
-				
-				long elapsed = System.nanoTime() - start;
-				System.out.println("Took: " + (elapsed / 1000000) + " msecs");
-				timesAvro[i] = elapsed;
-			}
-			
-			System.gc();
-			
-			System.out.println("Kryo serializer");
-			{
-				final TestDataOutputSerializer outView = new TestDataOutputSerializer(100000000);
-				ExecutionConfig conf = new ExecutionConfig();
-				conf.registerKryoType(MyType.class);
-				conf.enableForceKryo();
-				TypeInformation<MyType> typeInfo = new GenericTypeInfo<MyType>(MyType.class);
-				final TypeSerializer<MyType> serializer = typeInfo.createSerializer(conf);
-
-				long start = System.nanoTime();
-				
-				for (int k = 0; k < NUM_ELEMENTS; k++) {
-					serializer.serialize(elements[k], outView);
-				}
-				
-				final DataInputDeserializer inView = new DataInputDeserializer(outView.wrapAsByteBuffer());
-				for (int k = 0; k < NUM_ELEMENTS; k++) {
-					serializer.deserialize(dummy, inView);
-				}
-				
-				long elapsed = System.nanoTime() - start;
-				System.out.println("Took: " + (elapsed / 1000000) + " msecs");
-				timesKryo[i] = elapsed;
-			}
-		}
-	}
-	
-	
-	
-	
-	
-	public static class MyType {
-		
-		private String theString;
-		
-//		private Tuple2<Long, Double> theTuple;
-		
-		private List<Integer> theList;
-
-		
-		public MyType() {
-			theString = "";
-//			theTuple = new Tuple2<Long, Double>(0L, 0.0);
-			theList = new ArrayList<Integer>();
-		}
-		
-		public MyType(String theString, Tuple2<Long, Double> theTuple, List<Integer> theList) {
-			this.theString = theString;
-//			this.theTuple = theTuple;
-			this.theList = theList;
-		}
-
-		
-		public String getTheString() {
-			return theString;
-		}
-
-		public void setTheString(String theString) {
-			this.theString = theString;
-		}
-
-//		public Tuple2<Long, Double> getTheTuple() {
-//			return theTuple;
-//		}
-//
-//		public void setTheTuple(Tuple2<Long, Double> theTuple) {
-//			this.theTuple = theTuple;
-//		}
-
-		public List<Integer> getTheList() {
-			return theList;
-		}
-
-		public void setTheList(List<Integer> theList) {
-			this.theList = theList;
-		}
-		
-		
-		public static MyType getRandom() {
-			final int numListElements = rnd.nextInt(20);
-			List<Integer> list = new ArrayList<Integer>(numListElements);
-			for (int i = 0; i < numListElements; i++) {
-				list.add(rnd.nextInt());
-			}
-			
-			return new MyType(randomString(), new Tuple2<Long, Double>(rnd.nextLong(), rnd.nextDouble()), list);
-		}
-	}
-	
-	
-	private static String randomString() {
-		final int len = rnd.nextInt(100) + 20;
-		
-		StringBuilder bld = new StringBuilder();
-		for (int i = 0; i < len; i++) {
-			bld.append(rnd.nextInt('z' - 'a' + 1) + 'a');
-		}
-		return bld.toString();
-	}
-	
-	// ============================================================================================
-	// ============================================================================================
-	
-	public static final class DataInputDeserializer implements DataInputView {
-		
-		private byte[] buffer;
-		
-		private int end;
-
-		private int position;
-
-		public DataInputDeserializer() {
-		}
-		
-		public DataInputDeserializer(byte[] buffer, int start, int len) {
-			setBuffer(buffer, start, len);
-		}
-		
-		public DataInputDeserializer(ByteBuffer buffer) {
-			setBuffer(buffer);
-		}
-
-		public void setBuffer(ByteBuffer buffer) {
-			if (buffer.hasArray()) {
-				this.buffer = buffer.array();
-				this.position = buffer.arrayOffset() + buffer.position();
-				this.end = this.position + buffer.remaining();
-			} else if (buffer.isDirect()) {
-				this.buffer = new byte[buffer.remaining()];
-				this.position = 0;
-				this.end = this.buffer.length;
-
-				buffer.get(this.buffer);
-			} else {
-				throw new IllegalArgumentException("The given buffer is neither an array-backed heap ByteBuffer, nor a direct ByteBuffer.");
-			}
-		}
-
-		public void setBuffer(byte[] buffer, int start, int len) {
-			if (buffer == null) {
-				throw new NullPointerException();
-			}
-
-			if (start < 0 || len < 0 || start + len >= buffer.length) {
-				throw new IllegalArgumentException();
-			}
-
-			this.buffer = buffer;
-			this.position = start;
-			this.end = start * len;
-		}
-
-		// ----------------------------------------------------------------------------------------
-		//                               Data Input
-		// ----------------------------------------------------------------------------------------
-		
-		@Override
-		public boolean readBoolean() throws IOException {
-			if (this.position < this.end) {
-				return this.buffer[this.position++] != 0;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public byte readByte() throws IOException {
-			if (this.position < this.end) {
-				return this.buffer[this.position++];
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public char readChar() throws IOException {
-			if (this.position < this.end - 1) {
-				return (char) (((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0));
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public double readDouble() throws IOException {
-			return Double.longBitsToDouble(readLong());
-		}
-
-		@Override
-		public float readFloat() throws IOException {
-			return Float.intBitsToFloat(readInt());
-		}
-
-		@Override
-		public void readFully(byte[] b) throws IOException {
-			readFully(b, 0, b.length);
-		}
-
-		@Override
-		public void readFully(byte[] b, int off, int len) throws IOException {
-			if (len >= 0) {
-				if (off <= b.length - len) {
-					if (this.position <= this.end - len) {
-						System.arraycopy(this.buffer, position, b, off, len);
-						position += len;
-					} else {
-						throw new EOFException();
-					}
-				} else {
-					throw new ArrayIndexOutOfBoundsException();
-				}
-			} else if (len < 0) {
-				throw new IllegalArgumentException("Length may not be negative.");
-			}
-		}
-
-		@Override
-		public int readInt() throws IOException {
-			if (this.position >= 0 && this.position < this.end - 3) {
-				@SuppressWarnings("restriction")
-				int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
-				if (LITTLE_ENDIAN) {
-					value = Integer.reverseBytes(value);
-				}
-				
-				this.position += 4;
-				return value;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public String readLine() throws IOException {
-			if (this.position < this.end) {
-				// read until a newline is found
-				StringBuilder bld = new StringBuilder();
-				char curr = (char) readUnsignedByte();
-				while (position < this.end && curr != '\n') {
-					bld.append(curr);
-					curr = (char) readUnsignedByte();
-				}
-				// trim a trailing carriage return
-				int len = bld.length();
-				if (len > 0 && bld.charAt(len - 1) == '\r') {
-					bld.setLength(len - 1);
-				}
-				String s = bld.toString();
-				bld.setLength(0);
-				return s;
-			} else {
-				return null;
-			}
-		}
-
-		@Override
-		public long readLong() throws IOException {
-			if (position >= 0 && position < this.end - 7) {
-				@SuppressWarnings("restriction")
-				long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
-				if (LITTLE_ENDIAN) {
-					value = Long.reverseBytes(value);
-				}
-				this.position += 8;
-				return value;
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public short readShort() throws IOException {
-			if (position >= 0 && position < this.end - 1) {
-				return (short) ((((this.buffer[position++]) & 0xff) << 8) | (((this.buffer[position++]) & 0xff) << 0));
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public String readUTF() throws IOException {
-			int utflen = readUnsignedShort();
-			byte[] bytearr = new byte[utflen];
-			char[] chararr = new char[utflen];
-
-			int c, char2, char3;
-			int count = 0;
-			int chararr_count = 0;
-
-			readFully(bytearr, 0, utflen);
-
-			while (count < utflen) {
-				c = (int) bytearr[count] & 0xff;
-				if (c > 127) {
-					break;
-				}
-				count++;
-				chararr[chararr_count++] = (char) c;
-			}
-
-			while (count < utflen) {
-				c = (int) bytearr[count] & 0xff;
-				switch (c >> 4) {
-				case 0:
-				case 1:
-				case 2:
-				case 3:
-				case 4:
-				case 5:
-				case 6:
-				case 7:
-					/* 0xxxxxxx */
-					count++;
-					chararr[chararr_count++] = (char) c;
-					break;
-				case 12:
-				case 13:
-					/* 110x xxxx 10xx xxxx */
-					count += 2;
-					if (count > utflen) {
-						throw new UTFDataFormatException("malformed input: partial character at end");
-					}
-					char2 = (int) bytearr[count - 1];
-					if ((char2 & 0xC0) != 0x80) {
-						throw new UTFDataFormatException("malformed input around byte " + count);
-					}
-					chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
-					break;
-				case 14:
-					/* 1110 xxxx 10xx xxxx 10xx xxxx */
-					count += 3;
-					if (count > utflen) {
-						throw new UTFDataFormatException("malformed input: partial character at end");
-					}
-					char2 = (int) bytearr[count - 2];
-					char3 = (int) bytearr[count - 1];
-					if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
-						throw new UTFDataFormatException("malformed input around byte " + (count - 1));
-					}
-					chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
-					break;
-				default:
-					/* 10xx xxxx, 1111 xxxx */
-					throw new UTFDataFormatException("malformed input around byte " + count);
-				}
-			}
-			// The number of chars produced may be less than utflen
-			return new String(chararr, 0, chararr_count);
-		}
-
-		@Override
-		public int readUnsignedByte() throws IOException {
-			if (this.position < this.end) {
-				return (this.buffer[this.position++] & 0xff);
-			} else {
-				throw new EOFException();
-			}
-		}
-
-		@Override
-		public int readUnsignedShort() throws IOException {
-			if (this.position < this.end - 1) {
-				return ((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0);
-			} else {
-				throw new EOFException();
-			}
-		}
-		
-		@Override
-		public int skipBytes(int n) throws IOException {
-			if (this.position <= this.end - n) {
-				this.position += n;
-				return n;
-			} else {
-				n = this.end - this.position;
-				this.position = this.end;
-				return n;
-			}
-		}
-		
-		@SuppressWarnings("restriction")
-		private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
-		
-		@SuppressWarnings("restriction")
-		private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
-		
-		private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
-
-		@Override
-		public void skipBytesToRead(int numBytes) throws IOException {
-			int skippedBytes = skipBytes(numBytes);
-
-			if (skippedBytes < numBytes){
-				throw new EOFException("Could not skip " + numBytes +" bytes.");
-			}
-		}
-
-		@Override
-		public int read(byte[] b, int off, int len) throws IOException {
-			if (b == null) {
-				throw new NullPointerException("Byte array b cannot be null.");
-			}
-
-			if (off < 0) {
-				throw new IndexOutOfBoundsException("Offset cannot be negative.");
-			}
-
-			if (len < 0) {
-				throw new IndexOutOfBoundsException("Length cannot be negative.");
-			}
-
-			if (b.length - off < len) {
-				throw new IndexOutOfBoundsException("Byte array does not provide enough space to store requested data" +
-						".");
-			}
-
-			if (this.position >= this.end) {
-				return -1;
-			} else {
-				int toRead = Math.min(this.end-this.position, len);
-				System.arraycopy(this.buffer,this.position,b,off,toRead);
-				this.position += toRead;
-
-				return toRead;
-			}
-		}
-
-		@Override
-		public int read(byte[] b) throws IOException {
-			return read(b, 0, b.length);
-		}
-	}
-}


[05/13] flink git commit: [FLINK-2932] Examples in docs now download shell script using https instead of http

Posted by fh...@apache.org.
[FLINK-2932] Examples in docs now download shell script using https instead of http

This closes #1309


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

Branch: refs/heads/master
Commit: ec7bf50da0a1ec5ef52fd234fa46e3a00b951b92
Parents: 2ff04ba
Author: Frederick F. Kautz IV <fk...@alumni.cmu.edu>
Authored: Wed Oct 28 14:02:47 2015 -0700
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Oct 29 10:34:43 2015 +0100

----------------------------------------------------------------------
 docs/apis/example_connectors.md         | 2 +-
 docs/quickstart/java_api_quickstart.md  | 2 +-
 docs/quickstart/scala_api_quickstart.md | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ec7bf50d/docs/apis/example_connectors.md
----------------------------------------------------------------------
diff --git a/docs/apis/example_connectors.md b/docs/apis/example_connectors.md
index ef5e994..1a66529 100644
--- a/docs/apis/example_connectors.md
+++ b/docs/apis/example_connectors.md
@@ -145,7 +145,7 @@ Execute the following commands:
 2. Setup a new Flink project using the quickstarts:
 
    ~~~bash
-   curl http://flink.apache.org/q/quickstart.sh | bash
+   curl https://flink.apache.org/q/quickstart.sh | bash
    ~~~
 
 3. Add the following dependencies (in the `<dependencies>` section) to your `pom.xml` file:

http://git-wip-us.apache.org/repos/asf/flink/blob/ec7bf50d/docs/quickstart/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md
index 5be21ca..4d94396 100644
--- a/docs/quickstart/java_api_quickstart.md
+++ b/docs/quickstart/java_api_quickstart.md
@@ -41,7 +41,7 @@ Use one of the following commands to __create a project__:
 <div class="tab-content">
     <div class="tab-pane active" id="quickstart-script">
     {% highlight bash %}
-    $ curl http://flink.apache.org/q/quickstart.sh | bash
+    $ curl https://flink.apache.org/q/quickstart.sh | bash
     {% endhighlight %}
     </div>
     <div class="tab-pane" id="maven-archetype">

http://git-wip-us.apache.org/repos/asf/flink/blob/ec7bf50d/docs/quickstart/scala_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md
index 002488f..63f4d55 100644
--- a/docs/quickstart/scala_api_quickstart.md
+++ b/docs/quickstart/scala_api_quickstart.md
@@ -42,7 +42,7 @@ Use one of the following commands to __create a project__:
 <div class="tab-content">
     <div class="tab-pane active" id="quickstart-script">
 {% highlight bash %}
-$ curl http://flink.apache.org/q/quickstart-scala.sh | bash
+$ curl https://flink.apache.org/q/quickstart-scala.sh | bash
 {% endhighlight %}
     </div>
     <div class="tab-pane" id="maven-archetype">