You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2017/07/31 11:59:08 UTC
[01/12] flink git commit: [FLINK-7192] [java] Activate checkstyle
flink-java/test/operator
Repository: flink
Updated Branches:
refs/heads/master 80468b15c -> c5b61063e
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
index b7f7555..4fc14b5 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
@@ -29,6 +29,7 @@ import org.apache.flink.api.java.operators.ReduceOperator;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Test;
import java.util.ArrayList;
@@ -36,13 +37,16 @@ import java.util.List;
import static org.junit.Assert.assertTrue;
+/**
+ * Tests for {@link DataSet#reduce(ReduceFunction)}.
+ */
@SuppressWarnings("serial")
public class ReduceOperatorTest {
private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -50,10 +54,10 @@ public class ReduceOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
@Test
public void testSemanticPropsWithKeySelector1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -190,7 +194,7 @@ public class ReduceOperatorTest {
assertTrue(semProps.getReadFields(0) == null);
}
- public static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
+ private static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
@Override
public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, String, Long, Integer> value) throws Exception {
return new Tuple2<Long, Integer>();
@@ -199,7 +203,7 @@ public class ReduceOperatorTest {
@FunctionAnnotation.ForwardedFields("0->4;1;1->3;2")
@FunctionAnnotation.ReadFields("0;3;4")
- public static class DummyReduceFunction1 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyReduceFunction1 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public Tuple5<Integer, Long, String, Long, Integer> reduce(Tuple5<Integer, Long, String, Long, Integer> v1,
Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
@@ -208,27 +212,27 @@ public class ReduceOperatorTest {
}
@FunctionAnnotation.ReadFields("0;3;4")
- public static class DummyReduceFunction2 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyReduceFunction2 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public Tuple5<Integer, Long, String, Long, Integer> reduce(Tuple5<Integer, Long, String, Long, Integer> v1,
- Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
+ Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
return new Tuple5<Integer, Long, String, Long, Integer>();
}
}
- public static class DummyReduceFunction3 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyReduceFunction3 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public Tuple5<Integer, Long, String, Long, Integer> reduce(Tuple5<Integer, Long, String, Long, Integer> v1,
- Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
+ Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
return new Tuple5<Integer, Long, String, Long, Integer>();
}
}
@FunctionAnnotation.NonForwardedFields("2;4")
- public static class DummyReduceFunction4 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyReduceFunction4 implements ReduceFunction<Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public Tuple5<Integer, Long, String, Long, Integer> reduce(Tuple5<Integer, Long, String, Long, Integer> v1,
- Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
+ Tuple5<Integer, Long, String, Long, Integer> v2) throws Exception {
return new Tuple5<Integer, Long, String, Long, Integer>();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
index 709d830..389eb90 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/RightOuterJoinOperatorTest.java
@@ -28,11 +28,15 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
+/**
+ * Tests for {@link DataSet#rightOuterJoin(DataSet)}.
+ */
public class RightOuterJoinOperatorTest {
// TUPLE DATA
@@ -201,7 +205,6 @@ public class RightOuterJoinOperatorTest {
this.testRightOuterStrategies(JoinHint.BROADCAST_HASH_FIRST);
}
-
private void testRightOuterStrategies(JoinHint hint) {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -214,13 +217,12 @@ public class RightOuterJoinOperatorTest {
.with(new DummyJoin());
}
-
/*
* ####################################################################
*/
@SuppressWarnings("serial")
- public static class DummyJoin implements
+ private static class DummyJoin implements
JoinFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>, Long> {
@Override
@@ -230,7 +232,7 @@ public class RightOuterJoinOperatorTest {
}
@SuppressWarnings("serial")
- public static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
+ private static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
@Override
public Integer getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
@@ -239,7 +241,7 @@ public class RightOuterJoinOperatorTest {
}
@SuppressWarnings("serial")
- public static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
+ private static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
@Override
public Long getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
index c3307ec..4c4ca8d 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
@@ -30,6 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
+
import org.junit.Assert;
import org.junit.Test;
@@ -37,6 +38,9 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
+/**
+ * Tests for {@link DataSet#sortPartition(int, Order)}.
+ */
public class SortPartitionTest {
// TUPLE DATA
@@ -68,7 +72,6 @@ public class SortPartitionTest {
private final List<Tuple4<Integer, Long, CustomType, Long[]>> tupleWithCustomData =
new ArrayList<Tuple4<Integer, Long, CustomType, Long[]>>();
-
@Test
public void testSortPartitionPositionKeys1() {
@@ -78,7 +81,7 @@ public class SortPartitionTest {
// should work
try {
tupleDs.sortPartition(0, Order.ASCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -94,7 +97,7 @@ public class SortPartitionTest {
tupleDs
.sortPartition(0, Order.ASCENDING)
.sortPartition(3, Order.DESCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -128,7 +131,7 @@ public class SortPartitionTest {
// should work
try {
tupleDs.sortPartition("f1", Order.ASCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -144,7 +147,7 @@ public class SortPartitionTest {
tupleDs
.sortPartition("f0", Order.ASCENDING)
.sortPartition("f2.nested.myInt", Order.DESCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -224,7 +227,7 @@ public class SortPartitionTest {
// should work
try {
- tupleDs.sortPartition(new KeySelector<Tuple4<Integer,Long,CustomType,Long[]>, Tuple2<Integer, Long>>() {
+ tupleDs.sortPartition(new KeySelector<Tuple4<Integer, Long, CustomType, Long[]>, Tuple2<Integer, Long>>() {
@Override
public Tuple2<Integer, Long> getKey(Tuple4<Integer, Long, CustomType, Long[]> value) throws Exception {
return new Tuple2<>(value.f0, value.f1);
@@ -251,18 +254,25 @@ public class SortPartitionTest {
.sortPartition("f1", Order.ASCENDING);
}
+ /**
+ * Custom data type, for testing purposes.
+ */
public static class CustomType implements Serializable {
-
+
+ /**
+ * Custom nested data type, for testing purposes.
+ */
public static class Nest {
public int myInt;
}
+
private static final long serialVersionUID = 1L;
-
+
public int myInt;
public long myLong;
public String myString;
public Nest nested;
-
+
public CustomType() {}
public CustomType(int i, long l, String s) {
@@ -270,17 +280,11 @@ public class SortPartitionTest {
myLong = l;
myString = s;
}
-
+
@Override
public String toString() {
- return myInt+","+myLong+","+myString;
+ return myInt + "," + myLong + "," + myString;
}
}
- public static class CustomType2 implements Serializable {
-
- public int myInt;
- public int[] myIntArray;
-
- }
}
[05/12] flink git commit: [FLINK-7188] [java] Activate checkstyle
flink-java/summarize
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java
index ebbf627..fe29582 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregatorTest.java
@@ -19,10 +19,13 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
-
+/**
+ * Tests for {@link ShortSummaryAggregator}.
+ */
public class ShortSummaryAggregatorTest {
@Test
@@ -70,7 +73,7 @@ public class ShortSummaryAggregatorTest {
@Test
public void testMax() throws Exception {
Assert.assertEquals(1001, summarize(-1000, 0, 1, 50, 999, 1001).getMax().shortValue());
- Assert.assertEquals(0, summarize((int)Short.MIN_VALUE, -1000, 0).getMax().shortValue());
+ Assert.assertEquals(0, summarize((int) Short.MIN_VALUE, -1000, 0).getMax().shortValue());
Assert.assertEquals(11, summarize(1, 8, 7, 6, 9, 10, 2, 3, 5, 0, 11, -2, 3).getMax().shortValue());
Assert.assertEquals(11, summarize(1, 8, 7, 6, 9, null, 10, 2, 3, 5, null, 0, 11, -2, 3).getMax().shortValue());
Assert.assertNull(summarize().getMax());
@@ -79,46 +82,46 @@ public class ShortSummaryAggregatorTest {
@Test
public void testMin() throws Exception {
Assert.assertEquals(-1000, summarize(-1000, 0, 1, 50, 999, 1001).getMin().shortValue());
- Assert.assertEquals(Short.MIN_VALUE, summarize((int)Short.MIN_VALUE, -1000, 0).getMin().shortValue());
+ Assert.assertEquals(Short.MIN_VALUE, summarize((int) Short.MIN_VALUE, -1000, 0).getMin().shortValue());
Assert.assertEquals(-2, summarize(1, 8, 7, 6, 9, 10, 2, 3, 5, 0, 11, -2, 3).getMin().shortValue());
Assert.assertEquals(-2, summarize(1, 8, 7, 6, 9, null, 10, 2, 3, 5, null, 0, 11, -2, 3).getMin().shortValue());
Assert.assertNull(summarize().getMin());
}
/**
- * Helper method for summarizing a list of values
+ * Helper method for summarizing a list of values.
*/
protected NumericColumnSummary<Short> summarize(Integer... values) {
// cast everything to short here
Short[] shortValues = new Short[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
shortValues[i] = values[i].shortValue();
}
}
- return new AggregateCombineHarness<Short,NumericColumnSummary<Short>,ShortSummaryAggregator>() {
+ return new AggregateCombineHarness<Short, NumericColumnSummary<Short>, ShortSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Short> result1, NumericColumnSummary<Short> result2) {
- Assert.assertEquals(result1.getTotalCount(),result2.getTotalCount());
- Assert.assertEquals(result1.getNullCount(),result2.getNullCount());
- Assert.assertEquals(result1.getMissingCount(),result2.getMissingCount());
- Assert.assertEquals(result1.getNonMissingCount(),result2.getNonMissingCount());
- Assert.assertEquals(result1.getInfinityCount(),result2.getInfinityCount());
- Assert.assertEquals(result1.getNanCount(),result2.getNanCount());
+ Assert.assertEquals(result1.getTotalCount(), result2.getTotalCount());
+ Assert.assertEquals(result1.getNullCount(), result2.getNullCount());
+ Assert.assertEquals(result1.getMissingCount(), result2.getMissingCount());
+ Assert.assertEquals(result1.getNonMissingCount(), result2.getNonMissingCount());
+ Assert.assertEquals(result1.getInfinityCount(), result2.getInfinityCount());
+ Assert.assertEquals(result1.getNanCount(), result2.getNanCount());
- Assert.assertEquals(result1.containsNull(),result2.containsNull());
- Assert.assertEquals(result1.containsNonNull(),result2.containsNonNull());
+ Assert.assertEquals(result1.containsNull(), result2.containsNull());
+ Assert.assertEquals(result1.containsNonNull(), result2.containsNonNull());
- Assert.assertEquals(result1.getMin().shortValue(),result2.getMin().shortValue());
+ Assert.assertEquals(result1.getMin().shortValue(), result2.getMin().shortValue());
Assert.assertEquals(result1.getMax().shortValue(), result2.getMax().shortValue());
- Assert.assertEquals(result1.getSum().shortValue(),result2.getSum().shortValue());
+ Assert.assertEquals(result1.getSum().shortValue(), result2.getSum().shortValue());
Assert.assertEquals(result1.getMean().doubleValue(), result2.getMean().doubleValue(), 1e-12d);
- Assert.assertEquals(result1.getVariance().doubleValue(),result2.getVariance().doubleValue(), 1e-9d);
- Assert.assertEquals(result1.getStandardDeviation().doubleValue(),result2.getStandardDeviation().doubleValue(), 1e-12d);
+ Assert.assertEquals(result1.getVariance().doubleValue(), result2.getVariance().doubleValue(), 1e-9d);
+ Assert.assertEquals(result1.getStandardDeviation().doubleValue(), result2.getStandardDeviation().doubleValue(), 1e-12d);
}
}.summarize(shortValues);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java
index 8a8e7aa..6e60aea 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/ShortValueSummaryAggregatorTest.java
@@ -20,43 +20,47 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
import org.apache.flink.types.ShortValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.ShortValueSummaryAggregator}.
+ */
public class ShortValueSummaryAggregatorTest extends ShortSummaryAggregatorTest {
/**
- * Helper method for summarizing a list of values
+ * Helper method for summarizing a list of values.
*/
protected NumericColumnSummary<Short> summarize(Integer... values) {
ShortValue[] shortValues = new ShortValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
shortValues[i] = new ShortValue(values[i].shortValue());
}
}
- return new AggregateCombineHarness<ShortValue,NumericColumnSummary<Short>,ValueSummaryAggregator.ShortValueSummaryAggregator>() {
+ return new AggregateCombineHarness<ShortValue, NumericColumnSummary<Short>, ValueSummaryAggregator.ShortValueSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Short> result1, NumericColumnSummary<Short> result2) {
Assert.assertEquals(result1.getTotalCount(), result2.getTotalCount());
- Assert.assertEquals(result1.getNullCount(),result2.getNullCount());
- Assert.assertEquals(result1.getMissingCount(),result2.getMissingCount());
- Assert.assertEquals(result1.getNonMissingCount(),result2.getNonMissingCount());
- Assert.assertEquals(result1.getInfinityCount(),result2.getInfinityCount());
- Assert.assertEquals(result1.getNanCount(),result2.getNanCount());
+ Assert.assertEquals(result1.getNullCount(), result2.getNullCount());
+ Assert.assertEquals(result1.getMissingCount(), result2.getMissingCount());
+ Assert.assertEquals(result1.getNonMissingCount(), result2.getNonMissingCount());
+ Assert.assertEquals(result1.getInfinityCount(), result2.getInfinityCount());
+ Assert.assertEquals(result1.getNanCount(), result2.getNanCount());
- Assert.assertEquals(result1.containsNull(),result2.containsNull());
- Assert.assertEquals(result1.containsNonNull(),result2.containsNonNull());
+ Assert.assertEquals(result1.containsNull(), result2.containsNull());
+ Assert.assertEquals(result1.containsNonNull(), result2.containsNonNull());
- Assert.assertEquals(result1.getMin().shortValue(),result2.getMin().shortValue());
+ Assert.assertEquals(result1.getMin().shortValue(), result2.getMin().shortValue());
Assert.assertEquals(result1.getMax().shortValue(), result2.getMax().shortValue());
- Assert.assertEquals(result1.getSum().shortValue(),result2.getSum().shortValue());
+ Assert.assertEquals(result1.getSum().shortValue(), result2.getSum().shortValue());
Assert.assertEquals(result1.getMean().doubleValue(), result2.getMean().doubleValue(), 1e-12d);
- Assert.assertEquals(result1.getVariance().doubleValue(),result2.getVariance().doubleValue(), 1e-9d);
- Assert.assertEquals(result1.getStandardDeviation().doubleValue(),result2.getStandardDeviation().doubleValue(), 1e-12d);
+ Assert.assertEquals(result1.getVariance().doubleValue(), result2.getVariance().doubleValue(), 1e-9d);
+ Assert.assertEquals(result1.getStandardDeviation().doubleValue(), result2.getStandardDeviation().doubleValue(), 1e-12d);
}
}.summarize(shortValues);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java
index 02fc125..db12fac 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregatorTest.java
@@ -19,10 +19,13 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.StringColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
-
+/**
+ * Tests for {@link StringSummaryAggregator}.
+ */
public class StringSummaryAggregatorTest {
@Test
@@ -64,12 +67,12 @@ public class StringSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways.
*/
protected StringColumnSummary summarize(String... values) {
- return new AggregateCombineHarness<String,StringColumnSummary,StringSummaryAggregator>(){
+ return new AggregateCombineHarness<String, StringColumnSummary, StringSummaryAggregator>(){
@Override
protected void compareResults(StringColumnSummary result1, StringColumnSummary result2) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java
index 19bfd52..544e285 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/StringValueSummaryAggregatorTest.java
@@ -20,27 +20,31 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.StringColumnSummary;
import org.apache.flink.types.StringValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.StringValueSummaryAggregator}.
+ */
public class StringValueSummaryAggregatorTest extends StringSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways.
*/
@Override
protected StringColumnSummary summarize(String... values) {
StringValue[] stringValues = new StringValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
stringValues[i] = new StringValue(values[i]);
}
}
- return new AggregateCombineHarness<StringValue,StringColumnSummary,ValueSummaryAggregator.StringValueSummaryAggregator>(){
+ return new AggregateCombineHarness<StringValue, StringColumnSummary, ValueSummaryAggregator.StringValueSummaryAggregator>(){
@Override
protected void compareResults(StringColumnSummary result1, StringColumnSummary result2) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java
index 8134a90..bf3a100 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactoryTest.java
@@ -18,13 +18,22 @@
package org.apache.flink.api.java.summarize.aggregation;
-import org.apache.flink.types.*;
+import org.apache.flink.types.BooleanValue;
+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.ShortValue;
+import org.apache.flink.types.StringValue;
+
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
-
+/**
+ * Tests for {@link SummaryAggregatorFactory}.
+ */
public class SummaryAggregatorFactoryTest {
@Test
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/tools/maven/suppressions-java.xml
----------------------------------------------------------------------
diff --git a/tools/maven/suppressions-java.xml b/tools/maven/suppressions-java.xml
index 99aef69..6b63bb1 100644
--- a/tools/maven/suppressions-java.xml
+++ b/tools/maven/suppressions-java.xml
@@ -32,14 +32,6 @@ under the License.
checks="UnusedImports|AvoidStarImport"/>
<suppress
- files="(.*)api[/\\]java[/\\]summarize[/\\](.*)"
- checks="NewlineAtEndOfFile|RegexpSingleline|TodoComment|RedundantImport|ImportOrder|RedundantModifier|JavadocMethod|JavadocParagraph|JavadocType|JavadocStyle|PackageName|TypeNameCheck|ConstantNameCheck|StaticVariableNameCheck|MemberNameCheck|MethodNameCheck|ParameterName|LocalFinalVariableName|LocalVariableName|LeftCurly|UpperEll|FallThrough|reliefPattern|SimplifyBooleanExpression|EmptyStatement|ModifierOrder|EmptyLineSeparator|WhitespaceAround|WhitespaceAfter|NoWhitespaceAfter|NoWhitespaceBefore|OperatorWrap|ParenPad"/>
- <!--Only additional checks for test sources. Those checks were present in the "pre-strict" checkstyle but were not applied to test sources. We do not want to suppress them for sources directory-->
- <suppress
- files="(.*)test[/\\](.*)api[/\\]java[/\\]summarize[/\\](.*)"
- checks="AvoidStarImport"/>
-
- <suppress
files="(.*)api[/\\]java[/\\]aggregation[/\\](.*)"
checks="NewlineAtEndOfFile|RegexpSingleline|TodoComment|RedundantImport|ImportOrder|RedundantModifier|JavadocMethod|JavadocParagraph|JavadocType|JavadocStyle|PackageName|TypeNameCheck|ConstantNameCheck|StaticVariableNameCheck|MemberNameCheck|MethodNameCheck|ParameterName|LocalFinalVariableName|LocalVariableName|LeftCurly|UpperEll|FallThrough|reliefPattern|SimplifyBooleanExpression|EmptyStatement|ModifierOrder|EmptyLineSeparator|WhitespaceAround|WhitespaceAfter|NoWhitespaceAfter|NoWhitespaceBefore|OperatorWrap|ParenPad"/>
[12/12] flink git commit: [FLINK-7283][python] fix
PythonPlanBinderTest issues with python paths
Posted by ch...@apache.org.
[FLINK-7283][python] fix PythonPlanBinderTest issues with python paths
* the path is not set correctly (only inside config, not the configuration that
is passed on to the PythonPlanBinder
* linux distributions have become quite inventive regarding python binary names:
some offer python as Python 2, some as Python 3. Similarly, python3 and/or
python2 may not be available. If we really want to test both, we need to take
this into account.
This closes #4409.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c5b61063
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c5b61063
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c5b61063
Branch: refs/heads/master
Commit: c5b61063eebaaf46104e134a8536e2f660a3b429
Parents: d4b7ff3
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Thu Jul 27 16:35:37 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:42:56 2017 +0200
----------------------------------------------------------------------
.../flink/python/api/PythonPlanBinderTest.java | 85 ++++++++++++++------
1 file changed, 59 insertions(+), 26 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c5b61063/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java
index 701ac73..2a19a5f 100644
--- a/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java
+++ b/flink-libraries/flink-python/src/test/java/org/apache/flink/python/api/PythonPlanBinderTest.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -20,7 +20,9 @@ import org.apache.flink.core.fs.Path;
import org.apache.flink.python.api.streaming.data.PythonStreamer;
import org.apache.flink.test.util.JavaProgramTestBase;
+import java.io.BufferedReader;
import java.io.IOException;
+import java.io.InputStreamReader;
import java.util.ArrayList;
import java.util.List;
@@ -55,50 +57,81 @@ public class PythonPlanBinderTest extends JavaProgramTestBase {
return files;
}
- private static boolean isPython2Supported() throws IOException {
+ /**
+ * Finds the python binary for the given version.
+ *
+ * @param possibleBinaries
+ * binaries to test for
+ * @param expectedVersionPrefix
+ * expected output prefix for <tt><binary> -V</tt>, e.g. <tt>"Python 2."</tt>
+ *
+ * @return python binary or <tt>null</tt> if not supported
+ *
+ * @throws IOException
+ * if the process to test for the binaries failed to exit properly
+ */
+ private static String getPythonPath(String[] possibleBinaries, String expectedVersionPrefix)
+ throws IOException {
Process process = null;
+ for (String python : possibleBinaries) {
+ try {
+ process = new ProcessBuilder(python, "-V").redirectErrorStream(true).start();
+ BufferedReader stdInput = new BufferedReader(new
+ InputStreamReader(process.getInputStream()));
- try {
- process = Runtime.getRuntime().exec("python");
- return true;
- } catch (IOException ex) {
- return false;
- } finally {
- if (process != null) {
- PythonStreamer.destroyProcess(process);
+ if (stdInput.readLine().startsWith(expectedVersionPrefix)) {
+ return python;
+ }
+ } catch (IOException ignored) {
+ } finally {
+ if (process != null) {
+ PythonStreamer.destroyProcess(process);
+ }
}
}
+ return null;
}
- private static boolean isPython3Supported() throws IOException {
- Process process = null;
+ /**
+ * Finds the binary that executes python2 programs.
+ *
+ * @return python2 binary or <tt>null</tt> if not supported
+ *
+ * @throws IOException
+ * if the process to test for the binaries failed to exit properly
+ */
+ private static String getPython2Path() throws IOException {
+ return getPythonPath(new String[] {"python2", "python"}, "Python 2.");
+ }
- try {
- process = Runtime.getRuntime().exec("python3");
- return true;
- } catch (IOException ex) {
- return false;
- } finally {
- if (process != null) {
- PythonStreamer.destroyProcess(process);
- }
- }
+ /**
+ * Finds the binary that executes python3 programs.
+ *
+ * @return python3 binary or <tt>null</tt> if not supported
+ *
+ * @throws IOException
+ * if the process to test for the binaries failed to exit properly
+ */
+ private static String getPython3Path() throws IOException {
+ return getPythonPath(new String[] {"python3", "python"}, "Python 3.");
}
@Override
protected void testProgram() throws Exception {
String utils = findUtilsFile();
- if (isPython2Supported()) {
+ String python2 = getPython2Path();
+ if (python2 != null) {
for (String file : findTestFiles()) {
Configuration configuration = new Configuration();
- config.setString(PythonOptions.PYTHON_BINARY_PATH, "python");
+ configuration.setString(PythonOptions.PYTHON_BINARY_PATH, python2);
new PythonPlanBinder(configuration).runPlan(new String[]{file, utils});
}
}
- if (isPython3Supported()) {
+ String python3 = getPython3Path();
+ if (python3 != null) {
for (String file : findTestFiles()) {
Configuration configuration = new Configuration();
- config.setString(PythonOptions.PYTHON_BINARY_PATH, "python3");
+ configuration.setString(PythonOptions.PYTHON_BINARY_PATH, python3);
new PythonPlanBinder(configuration).runPlan(new String[]{file, utils});
}
}
[09/12] flink git commit: [hotfix] [tests] fix
JobRetrievalITCase#testJobRetrieval() test job not running
Posted by ch...@apache.org.
[hotfix] [tests] fix JobRetrievalITCase#testJobRetrieval() test job not running
Although this test should only verify that ClusterClient#retrieveJob() is
working, the inability to run the invokable given to the job does at least print
an exception in the test output that may be confusing.
This closes #4417.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/738dbbe5
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/738dbbe5
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/738dbbe5
Branch: refs/heads/master
Commit: 738dbbe57b1476f4d24ba8863d021ece11a6f514
Parents: a4a4e0b
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Fri Jul 28 14:16:10 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:09 2017 +0200
----------------------------------------------------------------------
.../apache/flink/test/example/client/JobRetrievalITCase.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/738dbbe5/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
index 6ce4f76..39eeccb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java
@@ -132,7 +132,12 @@ public class JobRetrievalITCase extends TestLogger {
}
}
- private static class SemaphoreInvokable extends AbstractInvokable {
+ /**
+ * Invokable that waits on {@link #lock} to be released and finishes afterwards.
+ *
+ * <p>NOTE: needs to be <tt>public</tt> so that a task can be run with this!
+ */
+ public static class SemaphoreInvokable extends AbstractInvokable {
@Override
public void invoke() throws Exception {
[02/12] flink git commit: [FLINK-7192] [java] Activate checkstyle
flink-java/test/operator
Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
index 0246f60..18c14e1 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
@@ -34,6 +34,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
+
import org.apache.hadoop.io.Writable;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -45,14 +46,17 @@ import java.util.List;
import static org.junit.Assert.assertTrue;
+/**
+ * Tests for {@link DataSet#join(DataSet)}.
+ */
@SuppressWarnings("serial")
public class JoinOperatorTest {
// TUPLE DATA
- private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+ private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo =
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo =
new TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -61,7 +65,7 @@ public class JoinOperatorTest {
BasicTypeInfo.INT_TYPE_INFO
);
// TUPLE DATA with nested Tuple2
- private static final List<Tuple5<Tuple2<Integer, String>, Long, String, Long, Integer>> emptyNestedTupleData =
+ private static final List<Tuple5<Tuple2<Integer, String>, Long, String, Long, Integer>> emptyNestedTupleData =
new ArrayList<Tuple5<Tuple2<Integer, String>, Long, String, Long, Integer>>();
private final TupleTypeInfo<Tuple5<Tuple2<Integer, String>, Long, String, Long, Integer>> nestedTupleTypeInfo =
@@ -72,9 +76,9 @@ public class JoinOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
// TUPLE DATA with nested CustomType
- private static final List<Tuple5<CustomType, Long, String, Long, Integer>> emptyNestedCustomTupleData =
+ private static final List<Tuple5<CustomType, Long, String, Long, Integer>> emptyNestedCustomTupleData =
new ArrayList<Tuple5<CustomType, Long, String, Long, Integer>>();
private final TupleTypeInfo<Tuple5<CustomType, Long, String, Long, Integer>> nestedCustomTupleTypeInfo =
@@ -85,23 +89,22 @@ public class JoinOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
private static List<CustomTypeWithTuple> customTypeWithTupleData = new ArrayList<CustomTypeWithTuple>();
private static List<CustomType> customTypeData = new ArrayList<CustomType>();
-
+
private static List<NestedCustomType> customNestedTypeData = new ArrayList<NestedCustomType>();
-
-
+
@BeforeClass
public static void insertCustomData() {
customTypeData.add(new CustomType());
customTypeWithTupleData.add(new CustomTypeWithTuple());
customNestedTypeData.add(new NestedCustomType());
}
-
- @Test
+
+ @Test
public void testJoinKeyFields1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -109,14 +112,14 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where(0).equalTo(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyFields2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -124,21 +127,21 @@ public class JoinOperatorTest {
// should not work, incompatible join key types
ds1.join(ds2).where(0).equalTo(2);
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyFields3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should not work, incompatible number of join keys
- ds1.join(ds2).where(0,1).equalTo(2);
+ ds1.join(ds2).where(0, 1).equalTo(2);
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testJoinKeyFields4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -146,10 +149,10 @@ public class JoinOperatorTest {
// should not work, join key out of range
ds1.join(ds2).where(5).equalTo(0);
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testJoinKeyFields5() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -157,10 +160,10 @@ public class JoinOperatorTest {
// should not work, negative key field position
ds1.join(ds2).where(-1).equalTo(-1);
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyFields6() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -179,11 +182,11 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where("myInt").equalTo("myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyExpressionsNested() {
@@ -194,12 +197,10 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where("myInt").equalTo("myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
-
@Test(expected = InvalidProgramException.class)
public void testJoinKeyExpressions2() {
@@ -233,7 +234,7 @@ public class JoinOperatorTest {
// should not work, join key non-existent
ds1.join(ds2).where("myNonExistent").equalTo("myInt");
}
-
+
/**
* Test if mixed types of key selectors are properly working.
*/
@@ -249,12 +250,12 @@ public class JoinOperatorTest {
return value.myInt;
}
});
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyMixedKeySelectorTurned() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -267,12 +268,12 @@ public class JoinOperatorTest {
return value.myInt;
}
}).equalTo("myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyMixedTupleIndex() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -280,12 +281,12 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
try {
ds1.join(ds2).where("f0").equalTo(4);
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyNestedTuples() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -293,12 +294,12 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
try {
ds1.join(ds2).where("f0.f0").equalTo(4);
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyNestedTuplesWithCustom() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -307,12 +308,12 @@ public class JoinOperatorTest {
try {
TypeInformation<?> t = ds1.join(ds2).where("f0.myInt").equalTo(4).getType();
assertTrue("not a composite type", t instanceof CompositeType);
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyWithCustomContainingTuple0() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -320,12 +321,12 @@ public class JoinOperatorTest {
DataSet<CustomTypeWithTuple> ds2 = env.fromCollection(customTypeWithTupleData);
try {
ds1.join(ds2).where("intByString.f0").equalTo("myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyWithCustomContainingTuple1() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -333,25 +334,25 @@ public class JoinOperatorTest {
DataSet<CustomTypeWithTuple> ds2 = env.fromCollection(customTypeWithTupleData);
try {
ds1.join(ds2).where("nested.myInt").equalTo("intByString.f0");
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyWithCustomContainingTuple2() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomTypeWithTuple> ds1 = env.fromCollection(customTypeWithTupleData);
DataSet<CustomTypeWithTuple> ds2 = env.fromCollection(customTypeWithTupleData);
try {
- ds1.join(ds2).where("nested.myInt", "myInt", "intByString.f1").equalTo("intByString.f0","myInt", "myString");
- } catch(Exception e) {
+ ds1.join(ds2).where("nested.myInt", "myInt", "intByString.f1").equalTo("intByString.f0", "myInt", "myString");
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyNestedTuplesWrongType() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -359,7 +360,7 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
ds1.join(ds2).where("f0.f1").equalTo(4); // f0.f1 is a String
}
-
+
@Test
public void testJoinKeyMixedTupleIndexTurned() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -367,12 +368,12 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
try {
ds1.join(ds2).where(0).equalTo("f0");
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyMixedTupleIndexWrongType() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -380,7 +381,7 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
ds1.join(ds2).where("f0").equalTo(3); // 3 is of type long, so it should fail
}
-
+
@Test
public void testJoinKeyMixedTupleIndex2() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -388,7 +389,7 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
try {
ds1.join(ds2).where("myInt").equalTo(4);
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
@@ -407,7 +408,7 @@ public class JoinOperatorTest {
}
});
}
-
+
@Test
public void testJoinKeyExpressions1Nested() {
@@ -418,7 +419,7 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where("nested.myInt").equalTo("nested.myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
@@ -457,10 +458,9 @@ public class JoinOperatorTest {
ds1.join(ds2).where("nested.myNonExistent").equalTo("nested.myInt");
}
-
@Test
public void testJoinKeySelectors1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomType> ds1 = env.fromCollection(customTypeData);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -470,7 +470,7 @@ public class JoinOperatorTest {
ds1.join(ds2)
.where(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -479,32 +479,31 @@ public class JoinOperatorTest {
)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
}
}
);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyMixing1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomType> ds1 = env.fromCollection(customTypeData);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
-
// should work
try {
ds1.join(ds2)
.where(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -512,15 +511,15 @@ public class JoinOperatorTest {
}
)
.equalTo(3);
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinKeyMixing2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -531,21 +530,21 @@ public class JoinOperatorTest {
.where(3)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
}
}
);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyMixing3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -555,7 +554,7 @@ public class JoinOperatorTest {
.where(2)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -563,20 +562,20 @@ public class JoinOperatorTest {
}
);
}
-
+
@Test(expected = InvalidProgramException.class)
public void testJoinKeyMixing4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
// should not work, more than one key field position
ds1.join(ds2)
- .where(1,3)
+ .where(1, 3)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -656,10 +655,10 @@ public class JoinOperatorTest {
ds1.join(ds2).where("*").equalTo("*");
}
-
+
@Test
public void testJoinProjection1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -668,14 +667,14 @@ public class JoinOperatorTest {
try {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection21() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -684,34 +683,34 @@ public class JoinOperatorTest {
try {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: field index is out of bounds of input tuple
try {
ds1.join(ds2).where(0).equalTo(0).projectFirst(-1);
Assert.fail();
- } catch(IndexOutOfBoundsException iob) {
+ } catch (IndexOutOfBoundsException iob) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: field index is out of bounds of input tuple
try {
ds1.join(ds2).where(0).equalTo(0).project(9);
Assert.fail();
- } catch(IndexOutOfBoundsException iob) {
+ } catch (IndexOutOfBoundsException iob) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -719,15 +718,15 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where(0).equalTo(0)
- .projectFirst(0,3);
- } catch(Exception e) {
+ .projectFirst(0, 3);
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -737,14 +736,14 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(0)
.projectSecond(3);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -752,18 +751,18 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where(0).equalTo(0)
- .projectFirst(0,2)
- .projectSecond(1,4)
+ .projectFirst(0, 2)
+ .projectSecond(1, 4)
.projectFirst(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
-
+
@Test
public void testJoinProjection5() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -771,17 +770,17 @@ public class JoinOperatorTest {
// should work
try {
ds1.join(ds2).where(0).equalTo(0)
- .projectSecond(0,2)
- .projectFirst(1,4)
+ .projectSecond(0, 2)
+ .projectFirst(1, 4)
.projectFirst(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection6() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomType> ds1 = env.fromCollection(customTypeData);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -806,16 +805,16 @@ public class JoinOperatorTest {
)
.projectFirst()
.projectSecond();
- } catch(Exception e) {
+ } catch (Exception e) {
System.out.println("FAILED: " + e);
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection26() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomType> ds1 = env.fromCollection(customTypeData);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -840,16 +839,16 @@ public class JoinOperatorTest {
)
.projectFirst()
.projectSecond();
- } catch(Exception e) {
+ } catch (Exception e) {
System.out.println("FAILED: " + e);
e.printStackTrace();
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection7() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -858,15 +857,15 @@ public class JoinOperatorTest {
try {
ds1.join(ds2).where(0).equalTo(0)
.projectSecond()
- .projectFirst(1,4);
- } catch(Exception e) {
+ .projectFirst(1, 4);
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testJoinProjection27() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -875,15 +874,15 @@ public class JoinOperatorTest {
try {
ds1.join(ds2).where(0).equalTo(0)
.projectSecond()
- .projectFirst(1,4);
- } catch(Exception e) {
+ .projectFirst(1, 4);
+ } catch (Exception e) {
Assert.fail();
}
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection8() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -892,10 +891,10 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(5);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection28() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -904,10 +903,10 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(5);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection9() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -916,10 +915,10 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectSecond(5);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection29() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -928,9 +927,9 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectSecond(5);
}
-
+
public void testJoinProjection10() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -939,10 +938,10 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(2);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection30() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -951,9 +950,9 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectFirst(-1);
}
-
+
public void testJoinProjection11() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -962,9 +961,9 @@ public class JoinOperatorTest {
ds1.join(ds2).where(0).equalTo(0)
.projectSecond(2);
}
-
+
public void testJoinProjection12() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -974,10 +973,10 @@ public class JoinOperatorTest {
.projectSecond(2)
.projectFirst(1);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection13() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -987,10 +986,10 @@ public class JoinOperatorTest {
.projectSecond(0)
.projectFirst(5);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection33() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -1000,10 +999,10 @@ public class JoinOperatorTest {
.projectSecond(-1)
.projectFirst(3);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection14() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -1013,10 +1012,10 @@ public class JoinOperatorTest {
.projectFirst(0)
.projectSecond(5);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testJoinProjection34() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -1034,32 +1033,32 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
- JoinOperator<?,?,?> joinOp = tupleDs1.join(tupleDs2)
+ JoinOperator<?, ?, ?> joinOp = tupleDs1.join(tupleDs2)
.where(new DummyTestKeySelector()).equalTo(new DummyTestKeySelector())
.with(new DummyTestJoinFunction1());
SemanticProperties semProps = joinOp.getSemanticProperties();
assertTrue(semProps.getForwardingTargetFields(0, 0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,2).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(0,2).contains(4));
- assertTrue(semProps.getForwardingTargetFields(0,3).size() == 2);
- assertTrue(semProps.getForwardingTargetFields(0,3).contains(1));
- assertTrue(semProps.getForwardingTargetFields(0,3).contains(3));
- assertTrue(semProps.getForwardingTargetFields(0,4).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,6).size() == 0);
-
- assertTrue(semProps.getForwardingTargetFields(1,0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,2).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,3).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,4).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(1,4).contains(2));
- assertTrue(semProps.getForwardingTargetFields(1,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,6).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(1,6).contains(0));
+ assertTrue(semProps.getForwardingTargetFields(0, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 2).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(0, 2).contains(4));
+ assertTrue(semProps.getForwardingTargetFields(0, 3).size() == 2);
+ assertTrue(semProps.getForwardingTargetFields(0, 3).contains(1));
+ assertTrue(semProps.getForwardingTargetFields(0, 3).contains(3));
+ assertTrue(semProps.getForwardingTargetFields(0, 4).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 6).size() == 0);
+
+ assertTrue(semProps.getForwardingTargetFields(1, 0).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 2).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 3).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 4).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(1, 4).contains(2));
+ assertTrue(semProps.getForwardingTargetFields(1, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 6).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(1, 6).contains(0));
assertTrue(semProps.getReadFields(0).size() == 3);
assertTrue(semProps.getReadFields(0).contains(2));
@@ -1078,7 +1077,7 @@ public class JoinOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
- JoinOperator<?,?,?> joinOp = tupleDs1.join(tupleDs2)
+ JoinOperator<?, ?, ?> joinOp = tupleDs1.join(tupleDs2)
.where(new DummyTestKeySelector()).equalTo(new DummyTestKeySelector())
.with(new DummyTestJoinFunction2())
.withForwardedFieldsFirst("2;4->0")
@@ -1086,26 +1085,26 @@ public class JoinOperatorTest {
SemanticProperties semProps = joinOp.getSemanticProperties();
- assertTrue(semProps.getForwardingTargetFields(0,0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,2).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,3).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,4).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(0,4).contains(2));
- assertTrue(semProps.getForwardingTargetFields(0,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,6).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(0,6).contains(0));
-
- assertTrue(semProps.getForwardingTargetFields(1,0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,2).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(1,2).contains(4));
- assertTrue(semProps.getForwardingTargetFields(1,3).size() == 2);
- assertTrue(semProps.getForwardingTargetFields(1,3).contains(1));
- assertTrue(semProps.getForwardingTargetFields(1,3).contains(3));
- assertTrue(semProps.getForwardingTargetFields(1,4).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,6).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 0).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 2).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 3).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 4).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(0, 4).contains(2));
+ assertTrue(semProps.getForwardingTargetFields(0, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 6).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(0, 6).contains(0));
+
+ assertTrue(semProps.getForwardingTargetFields(1, 0).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 2).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(1, 2).contains(4));
+ assertTrue(semProps.getForwardingTargetFields(1, 3).size() == 2);
+ assertTrue(semProps.getForwardingTargetFields(1, 3).contains(1));
+ assertTrue(semProps.getForwardingTargetFields(1, 3).contains(3));
+ assertTrue(semProps.getForwardingTargetFields(1, 4).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 6).size() == 0);
assertTrue(semProps.getReadFields(0).size() == 3);
assertTrue(semProps.getReadFields(0).contains(2));
@@ -1160,6 +1159,9 @@ public class JoinOperatorTest {
* ####################################################################
*/
+ /**
+ * Custom type for testing.
+ */
public static class Nested implements Serializable {
private static final long serialVersionUID = 1L;
@@ -1174,10 +1176,13 @@ public class JoinOperatorTest {
@Override
public String toString() {
- return ""+myInt;
+ return "" + myInt;
}
}
- // a simple nested type (only basic types)
+
+ /**
+ * Simple nested type (only basic types).
+ */
public static class NestedCustomType implements Serializable {
private static final long serialVersionUID = 1L;
@@ -1186,8 +1191,9 @@ public class JoinOperatorTest {
public long myLong;
public String myString;
public Nested nest;
-
- public NestedCustomType() {}
+
+ public NestedCustomType() {
+ }
public NestedCustomType(int i, long l, String s) {
myInt = i;
@@ -1197,14 +1203,17 @@ public class JoinOperatorTest {
@Override
public String toString() {
- return myInt+","+myLong+","+myString+","+nest;
+ return myInt + "," + myLong + "," + myString + "," + nest;
}
}
+ /**
+ * Custom type for testing.
+ */
public static class CustomType implements Serializable {
-
+
private static final long serialVersionUID = 1L;
-
+
public int myInt;
public long myLong;
public NestedCustomType nested;
@@ -1212,8 +1221,9 @@ public class JoinOperatorTest {
public Object nothing;
public List<String> countries;
public Writable interfaceTest;
-
- public CustomType() {}
+
+ public CustomType() {
+ }
public CustomType(int i, long l, String s) {
myInt = i;
@@ -1223,25 +1233,28 @@ public class JoinOperatorTest {
interfaceTest = null;
nested = new NestedCustomType(i, l, s);
}
-
+
@Override
public String toString() {
- return myInt+","+myLong+","+myString;
+ return myInt + "," + myLong + "," + myString;
}
}
-
-
+
+ /**
+ * Custom type for testing.
+ */
public static class CustomTypeWithTuple implements Serializable {
-
+
private static final long serialVersionUID = 1L;
-
+
public int myInt;
public long myLong;
public NestedCustomType nested;
public String myString;
public Tuple2<Integer, String> intByString;
-
- public CustomTypeWithTuple() {}
+
+ public CustomTypeWithTuple() {
+ }
public CustomTypeWithTuple(int i, long l, String s) {
myInt = i;
@@ -1250,14 +1263,14 @@ public class JoinOperatorTest {
nested = new NestedCustomType(i, l, s);
intByString = new Tuple2<Integer, String>(i, s);
}
-
+
@Override
public String toString() {
- return myInt+","+myLong+","+myString;
+ return myInt + "," + myLong + "," + myString;
}
}
- public static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
+ private static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
@Override
public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, String, Long, Integer> value) throws Exception {
return new Tuple2<Long, Integer>();
@@ -1268,7 +1281,7 @@ public class JoinOperatorTest {
@FunctionAnnotation.ForwardedFieldsSecond("2;4->0")
@FunctionAnnotation.ReadFieldsFirst("0;2;4")
@FunctionAnnotation.ReadFieldsSecond("1;3")
- public static class DummyTestJoinFunction1
+ private static class DummyTestJoinFunction1
implements JoinFunction<Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>> {
@@ -1281,7 +1294,7 @@ public class JoinOperatorTest {
}
@FunctionAnnotation.ReadFieldsFirst("0;1;2")
- public static class DummyTestJoinFunction2
+ private static class DummyTestJoinFunction2
implements JoinFunction<Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>> {
@@ -1292,5 +1305,5 @@ public class JoinOperatorTest {
return new Tuple5<Integer, Long, String, Long, Integer>();
}
}
-
+
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
index bfcc3e8..b4790ac 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/LeftOuterJoinOperatorTest.java
@@ -19,21 +19,24 @@
package org.apache.flink.api.java.operator;
import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-
-import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeutils.CompositeType;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
+/**
+ * Tests for {@link DataSet#leftOuterJoin(DataSet)}.
+ */
public class LeftOuterJoinOperatorTest {
// TUPLE DATA
@@ -202,7 +205,6 @@ public class LeftOuterJoinOperatorTest {
this.testLeftOuterStrategies(JoinHint.BROADCAST_HASH_FIRST);
}
-
private void testLeftOuterStrategies(JoinHint hint) {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -215,13 +217,12 @@ public class LeftOuterJoinOperatorTest {
.with(new DummyJoin());
}
-
/*
* ####################################################################
*/
@SuppressWarnings("serial")
- public static class DummyJoin implements
+ private static class DummyJoin implements
JoinFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>, Long> {
@Override
@@ -231,7 +232,7 @@ public class LeftOuterJoinOperatorTest {
}
@SuppressWarnings("serial")
- public static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
+ private static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
@Override
public Integer getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
@@ -240,7 +241,7 @@ public class LeftOuterJoinOperatorTest {
}
@SuppressWarnings("serial")
- public static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
+ private static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
@Override
public Long getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
index cbb7690..b207e19 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
@@ -15,11 +15,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.flink.api.java.operator;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
+package org.apache.flink.api.java.operator;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -28,9 +25,17 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.operators.UnsortedGrouping;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Assert;
import org.junit.Test;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#maxBy(int...)}.
+ */
public class MaxByOperatorTest {
// TUPLE DATA
@@ -42,7 +47,7 @@ public class MaxByOperatorTest {
BasicTypeInfo.INT_TYPE_INFO);
/**
- * This test validates that no exceptions is thrown when an empty dataset
+ * This test validates that no exceptions is thrown when an empty dataset
* calls maxBy().
*/
@Test
@@ -62,23 +67,23 @@ public class MaxByOperatorTest {
}
private final List<CustomType> customTypeData = new ArrayList<CustomType>();
-
+
/**
- * This test validates that an InvalidProgrammException is thrown when maxBy
+ * This test validates that an InvalidProgrammException is thrown when maxBy
* is used on a custom data type.
*/
@Test(expected = InvalidProgramException.class)
public void testCustomKeyFieldsDataset() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should not work: groups on custom type
customDs.maxBy(0);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -92,7 +97,7 @@ public class MaxByOperatorTest {
// should not work, key out of tuple bounds
tupleDs.maxBy(5);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -106,7 +111,7 @@ public class MaxByOperatorTest {
// should not work, key out of tuple bounds
tupleDs.maxBy(-1);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -118,13 +123,13 @@ public class MaxByOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should not work, key out of tuple bounds
- tupleDs.maxBy(1,2,3,4,-1);
+ tupleDs.maxBy(1, 2, 3, 4, -1);
}
-
+
//---------------------------- GROUPING TESTS BELOW --------------------------------------
-
+
/**
- * This test validates that no exceptions is thrown when an empty grouping
+ * This test validates that no exceptions is thrown when an empty grouping
* calls maxBy().
*/
@Test
@@ -142,21 +147,21 @@ public class MaxByOperatorTest {
}
/**
- * This test validates that an InvalidProgrammException is thrown when maxBy
+ * This test validates that an InvalidProgrammException is thrown when maxBy
* is used on a custom data type.
*/
@Test(expected = InvalidProgramException.class)
public void testCustomKeyFieldsGrouping() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
UnsortedGrouping<CustomType> groupDs = env.fromCollection(customTypeData).groupBy(0);
// should not work: groups on custom type
groupDs.maxBy(0);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -170,7 +175,7 @@ public class MaxByOperatorTest {
// should not work, key out of tuple bounds
groupDs.maxBy(5);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -184,7 +189,7 @@ public class MaxByOperatorTest {
// should not work, key out of tuple bounds
groupDs.maxBy(-1);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -196,7 +201,7 @@ public class MaxByOperatorTest {
UnsortedGrouping<Tuple5<Integer, Long, String, Long, Integer>> groupDs = env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0);
// should not work, key out of tuple bounds
- groupDs.maxBy(1,2,3,4,-1);
+ groupDs.maxBy(1, 2, 3, 4, -1);
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
index b9659c0..02b84fa 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
@@ -15,13 +15,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.flink.api.java.operator;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
+package org.apache.flink.api.java.operator;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -30,8 +25,17 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.operators.UnsortedGrouping;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+import org.junit.Assert;
import org.junit.Test;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#minBy(int...)}.
+ */
public class MinByOperatorTest {
// TUPLE DATA
@@ -43,7 +47,7 @@ public class MinByOperatorTest {
BasicTypeInfo.INT_TYPE_INFO);
/**
- * This test validates that no exceptions is thrown when an empty dataset
+ * This test validates that no exceptions is thrown when an empty dataset
* calls minBy().
*/
@Test
@@ -63,23 +67,23 @@ public class MinByOperatorTest {
}
private final List<CustomType> customTypeData = new ArrayList<CustomType>();
-
+
/**
- * This test validates that an InvalidProgrammException is thrown when minBy
+ * This test validates that an InvalidProgrammException is thrown when minBy
* is used on a custom data type.
*/
@Test(expected = InvalidProgramException.class)
public void testCustomKeyFieldsDataset() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should not work: groups on custom type
customDs.minBy(0);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -93,7 +97,7 @@ public class MinByOperatorTest {
// should not work, key out of tuple bounds
tupleDs.minBy(5);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -107,7 +111,7 @@ public class MinByOperatorTest {
// should not work, key out of tuple bounds
tupleDs.minBy(-1);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -119,13 +123,13 @@ public class MinByOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should not work, key out of tuple bounds
- tupleDs.minBy(1,2,3,4,-1);
+ tupleDs.minBy(1, 2, 3, 4, -1);
}
-
+
//---------------------------- GROUPING TESTS BELOW --------------------------------------
-
+
/**
- * This test validates that no exceptions is thrown when an empty grouping
+ * This test validates that no exceptions is thrown when an empty grouping
* calls minBy().
*/
@Test
@@ -143,21 +147,21 @@ public class MinByOperatorTest {
}
/**
- * This test validates that an InvalidProgrammException is thrown when minBy
+ * This test validates that an InvalidProgrammException is thrown when minBy
* is used on a custom data type.
*/
@Test(expected = InvalidProgramException.class)
public void testCustomKeyFieldsGrouping() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
UnsortedGrouping<CustomType> groupDs = env.fromCollection(customTypeData).groupBy(0);
// should not work: groups on custom type
groupDs.minBy(0);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -171,7 +175,7 @@ public class MinByOperatorTest {
// should not work, key out of tuple bounds
groupDs.minBy(5);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -185,7 +189,7 @@ public class MinByOperatorTest {
// should not work, key out of tuple bounds
groupDs.minBy(-1);
}
-
+
/**
* This test validates that an index which is out of bounds throws an
* IndexOutOfBOundsExcpetion.
@@ -197,7 +201,7 @@ public class MinByOperatorTest {
UnsortedGrouping<Tuple5<Integer, Long, String, Long, Integer>> groupDs = env.fromCollection(emptyTupleData, tupleTypeInfo).groupBy(0);
// should not work, key out of tuple bounds
- groupDs.minBy(1,2,3,4,-1);
+ groupDs.minBy(1, 2, 3, 4, -1);
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
index 3de254b..a99b769 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java
@@ -23,12 +23,16 @@ import org.apache.flink.api.common.operators.ResourceSpec;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.operators.Operator;
import org.apache.flink.api.java.typeutils.ValueTypeInfo;
+
import org.junit.Test;
import java.lang.reflect.Method;
import static org.junit.Assert.assertEquals;
+/**
+ * Tests for {@link Operator}.
+ */
public class OperatorTest {
@Test
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java
index 34f72a2..be95db8 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/PartitionOperatorTest.java
@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.flink.api.java.operator;
import org.apache.flink.api.common.InvalidProgramException;
@@ -25,383 +26,384 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.operators.DataSource;
import org.apache.flink.api.java.tuple.Tuple2;
+
import org.junit.Test;
import java.io.Serializable;
+/**
+ * Tests for partitioning.
+ */
public class PartitionOperatorTest {
- public static class CustomPojo implements Serializable, Comparable<CustomPojo> {
- private Integer number;
- private String name;
-
- public CustomPojo() {
- }
-
- public CustomPojo(Integer number,
- String name) {
- this.number = number;
- this.name = name;
- }
-
- public Integer getNumber() {
- return number;
- }
-
- public void setNumber(Integer number) {
- this.number = number;
- }
-
- public String getName() {
- return name;
- }
-
- public void setName(String name) {
- this.name = name;
- }
-
- @Override
- public int compareTo(CustomPojo o) {
- return Integer.compare(this.number, o.number);
- }
- }
-
- public static class NestedPojo implements Serializable {
- private CustomPojo nested;
- private Long outer;
-
- public NestedPojo() {
- }
-
- public NestedPojo(CustomPojo nested,
- Long outer) {
- this.nested = nested;
- this.outer = outer;
- }
-
- public CustomPojo getNested() {
- return nested;
- }
-
- public void setNested(CustomPojo nested) {
- this.nested = nested;
- }
-
- public Long getOuter() {
- return outer;
- }
-
- public void setOuter(Long outer) {
- this.outer = outer;
- }
- }
-
- private DataSet<Tuple2<Integer, String>> getTupleDataSet(ExecutionEnvironment env) {
- return env.fromElements(
- new Tuple2<>(1, "first"),
- new Tuple2<>(2, "second"),
- new Tuple2<>(3, "third"),
- new Tuple2<>(4, "fourth"),
- new Tuple2<>(5, "fifth"),
- new Tuple2<>(6, "sixth")
- );
- }
-
- private DataSet<CustomPojo> getPojoDataSet(ExecutionEnvironment env) {
- return env.fromElements(
- new CustomPojo(1, "first"),
- new CustomPojo(2, "second"),
- new CustomPojo(3, "third"),
- new CustomPojo(4, "fourth"),
- new CustomPojo(5, "fifth"),
- new CustomPojo(6, "sixth")
- );
- }
-
- private DataSet<NestedPojo> getNestedPojoDataSet(ExecutionEnvironment env) {
- return env.fromElements(
- new NestedPojo(new CustomPojo(1, "first"), 1L),
- new NestedPojo(new CustomPojo(2, "second"), 2L),
- new NestedPojo(new CustomPojo(3, "third"), 3L),
- new NestedPojo(new CustomPojo(4, "fourth"), 4L),
- new NestedPojo(new CustomPojo(5, "fifth"), 5L),
- new NestedPojo(new CustomPojo(6, "sixth"), 6L)
- );
- }
-
- @Test
- public void testRebalance() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.rebalance();
- }
-
- @Test
- public void testHashPartitionByField1() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByHash(0);
- }
-
- @Test
- public void testHashPartitionByField2() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByHash(0, 1);
- }
-
- @Test(expected = IndexOutOfBoundsException.class)
- public void testHashPartitionByFieldOutOfRange() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByHash(0, 1, 2);
- }
-
- @Test
- public void testHashPartitionByFieldName1() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionByHash("number");
- }
-
- @Test
- public void testHashPartitionByFieldName2() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionByHash("number", "name");
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testHashPartitionByInvalidFieldName() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionByHash("number", "name", "invalidField");
- }
-
- @Test
- public void testRangePartitionByFieldName1() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionByRange("number");
- }
-
- @Test
- public void testRangePartitionByFieldName2() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionByRange("number", "name");
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testRangePartitionByInvalidFieldName() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionByRange("number", "name", "invalidField");
- }
-
- @Test
- public void testRangePartitionByField1() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByRange(0);
- }
-
- @Test
- public void testRangePartitionByField2() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByRange(0, 1);
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testRangePartitionWithEmptyIndicesKey() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSource<Tuple2<Tuple2<Integer, Integer>, Integer>> ds = env.fromElements(
- new Tuple2<>(new Tuple2<>(1, 1), 1),
- new Tuple2<>(new Tuple2<>(2, 2), 2),
- new Tuple2<>(new Tuple2<>(2, 2), 2)
- );
- ds.partitionByRange(new int[]{});
- }
-
- @Test(expected = IndexOutOfBoundsException.class)
- public void testRangePartitionByFieldOutOfRange() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByRange(0, 1, 2);
- }
-
- @Test(expected = IllegalStateException.class)
- public void testHashPartitionWithOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByHash(1).withOrders(Order.ASCENDING);
- }
-
- @Test(expected = IllegalStateException.class)
- public void testRebalanceWithOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.rebalance().withOrders(Order.ASCENDING);
- }
-
- @Test
- public void testRangePartitionWithOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByRange(0).withOrders(Order.ASCENDING);
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testRangePartitionWithTooManyOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionByRange(0).withOrders(Order.ASCENDING, Order.DESCENDING);
- }
-
- @Test
- public void testRangePartitionByComplexKeyWithOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSource<Tuple2<Tuple2<Integer, Integer>, Integer>> ds = env.fromElements(
- new Tuple2<>(new Tuple2<>(1, 1), 1),
- new Tuple2<>(new Tuple2<>(2, 2), 2),
- new Tuple2<>(new Tuple2<>(2, 2), 2)
- );
- ds.partitionByRange(0, 1).withOrders(Order.ASCENDING, Order.DESCENDING);
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testRangePartitionByComplexKeyWithTooManyOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSource<Tuple2<Tuple2<Integer, Integer>, Integer>> ds = env.fromElements(
- new Tuple2<>(new Tuple2<>(1, 1), 1),
- new Tuple2<>(new Tuple2<>(2, 2), 2),
- new Tuple2<>(new Tuple2<>(2, 2), 2)
- );
- ds.partitionByRange(0).withOrders(Order.ASCENDING, Order.DESCENDING);
- }
-
- @Test
- public void testRangePartitionBySelectorComplexKeyWithOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<NestedPojo> ds = getNestedPojoDataSet(env);
- ds.partitionByRange(new KeySelector<NestedPojo, CustomPojo>() {
- @Override
- public CustomPojo getKey(NestedPojo value) throws Exception {
- return value.getNested();
- }
- }).withOrders(Order.ASCENDING);
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testRangePartitionBySelectorComplexKeyWithTooManyOrders() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<NestedPojo> ds = getNestedPojoDataSet(env);
- ds.partitionByRange(new KeySelector<NestedPojo, CustomPojo>() {
- @Override
- public CustomPojo getKey(NestedPojo value) throws Exception {
- return value.getNested();
- }
- }).withOrders(Order.ASCENDING, Order.DESCENDING);
- }
-
- @Test
- public void testRangePartitionCustomPartitionerByFieldId() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionCustom(new Partitioner<Integer>() {
- @Override
- public int partition(Integer key,
- int numPartitions) {
- return 1;
- }
- }, 0);
- }
-
- @Test(expected = InvalidProgramException.class)
- public void testRangePartitionInvalidCustomPartitionerByFieldId() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
- ds.partitionCustom(new Partitioner<Integer>() {
- @Override
- public int partition(Integer key,
- int numPartitions) {
- return 1;
- }
- }, 1);
- }
-
- @Test
- public void testRangePartitionCustomPartitionerByFieldName() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionCustom(new Partitioner<Integer>() {
- @Override
- public int partition(Integer key,
- int numPartitions) {
- return 1;
- }
- }, "number");
- }
-
- @Test(expected = InvalidProgramException.class)
- public void testRangePartitionInvalidCustomPartitionerByFieldName() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionCustom(new Partitioner<Integer>() {
- @Override
- public int partition(Integer key,
- int numPartitions) {
- return 1;
- }
- }, "name");
- }
-
- @Test
- public void testRangePartitionCustomPartitionerByKeySelector() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- final DataSet<CustomPojo> ds = getPojoDataSet(env);
- ds.partitionCustom(new Partitioner<Integer>() {
- @Override
- public int partition(Integer key,
- int numPartitions) {
- return 1;
- }
- }, new KeySelector<CustomPojo, Integer>() {
- @Override
- public Integer getKey(CustomPojo value) throws Exception {
- return value.getNumber();
- }
- });
- }
-
-
+ /**
+ * Custom data type, for testing purposes.
+ */
+ public static class CustomPojo implements Serializable, Comparable<CustomPojo> {
+ private Integer number;
+ private String name;
+
+ public CustomPojo() {
+ }
+
+ public CustomPojo(Integer number, String name) {
+ this.number = number;
+ this.name = name;
+ }
+
+ public Integer getNumber() {
+ return number;
+ }
+
+ public void setNumber(Integer number) {
+ this.number = number;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public int compareTo(CustomPojo o) {
+ return Integer.compare(this.number, o.number);
+ }
+ }
+
+ /**
+ * Custom data type with nested type, for testing purposes.
+ */
+ public static class NestedPojo implements Serializable {
+ private CustomPojo nested;
+ private Long outer;
+
+ public NestedPojo() {
+ }
+
+ public NestedPojo(CustomPojo nested, Long outer) {
+ this.nested = nested;
+ this.outer = outer;
+ }
+
+ public CustomPojo getNested() {
+ return nested;
+ }
+
+ public void setNested(CustomPojo nested) {
+ this.nested = nested;
+ }
+
+ public Long getOuter() {
+ return outer;
+ }
+
+ public void setOuter(Long outer) {
+ this.outer = outer;
+ }
+ }
+
+ private DataSet<Tuple2<Integer, String>> getTupleDataSet(ExecutionEnvironment env) {
+ return env.fromElements(
+ new Tuple2<>(1, "first"),
+ new Tuple2<>(2, "second"),
+ new Tuple2<>(3, "third"),
+ new Tuple2<>(4, "fourth"),
+ new Tuple2<>(5, "fifth"),
+ new Tuple2<>(6, "sixth")
+ );
+ }
+
+ private DataSet<CustomPojo> getPojoDataSet(ExecutionEnvironment env) {
+ return env.fromElements(
+ new CustomPojo(1, "first"),
+ new CustomPojo(2, "second"),
+ new CustomPojo(3, "third"),
+ new CustomPojo(4, "fourth"),
+ new CustomPojo(5, "fifth"),
+ new CustomPojo(6, "sixth")
+ );
+ }
+
+ private DataSet<NestedPojo> getNestedPojoDataSet(ExecutionEnvironment env) {
+ return env.fromElements(
+ new NestedPojo(new CustomPojo(1, "first"), 1L),
+ new NestedPojo(new CustomPojo(2, "second"), 2L),
+ new NestedPojo(new CustomPojo(3, "third"), 3L),
+ new NestedPojo(new CustomPojo(4, "fourth"), 4L),
+ new NestedPojo(new CustomPojo(5, "fifth"), 5L),
+ new NestedPojo(new CustomPojo(6, "sixth"), 6L)
+ );
+ }
+
+ @Test
+ public void testRebalance() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.rebalance();
+ }
+
+ @Test
+ public void testHashPartitionByField1() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByHash(0);
+ }
+
+ @Test
+ public void testHashPartitionByField2() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByHash(0, 1);
+ }
+
+ @Test(expected = IndexOutOfBoundsException.class)
+ public void testHashPartitionByFieldOutOfRange() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByHash(0, 1, 2);
+ }
+
+ @Test
+ public void testHashPartitionByFieldName1() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionByHash("number");
+ }
+
+ @Test
+ public void testHashPartitionByFieldName2() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionByHash("number", "name");
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testHashPartitionByInvalidFieldName() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionByHash("number", "name", "invalidField");
+ }
+
+ @Test
+ public void testRangePartitionByFieldName1() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionByRange("number");
+ }
+
+ @Test
+ public void testRangePartitionByFieldName2() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionByRange("number", "name");
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testRangePartitionByInvalidFieldName() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionByRange("number", "name", "invalidField");
+ }
+
+ @Test
+ public void testRangePartitionByField1() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByRange(0);
+ }
+
+ @Test
+ public void testRangePartitionByField2() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByRange(0, 1);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testRangePartitionWithEmptyIndicesKey() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSource<Tuple2<Tuple2<Integer, Integer>, Integer>> ds = env.fromElements(
+ new Tuple2<>(new Tuple2<>(1, 1), 1),
+ new Tuple2<>(new Tuple2<>(2, 2), 2),
+ new Tuple2<>(new Tuple2<>(2, 2), 2)
+ );
+ ds.partitionByRange(new int[]{});
+ }
+
+ @Test(expected = IndexOutOfBoundsException.class)
+ public void testRangePartitionByFieldOutOfRange() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByRange(0, 1, 2);
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testHashPartitionWithOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByHash(1).withOrders(Order.ASCENDING);
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testRebalanceWithOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.rebalance().withOrders(Order.ASCENDING);
+ }
+
+ @Test
+ public void testRangePartitionWithOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByRange(0).withOrders(Order.ASCENDING);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testRangePartitionWithTooManyOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionByRange(0).withOrders(Order.ASCENDING, Order.DESCENDING);
+ }
+
+ @Test
+ public void testRangePartitionByComplexKeyWithOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSource<Tuple2<Tuple2<Integer, Integer>, Integer>> ds = env.fromElements(
+ new Tuple2<>(new Tuple2<>(1, 1), 1),
+ new Tuple2<>(new Tuple2<>(2, 2), 2),
+ new Tuple2<>(new Tuple2<>(2, 2), 2)
+ );
+ ds.partitionByRange(0, 1).withOrders(Order.ASCENDING, Order.DESCENDING);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testRangePartitionByComplexKeyWithTooManyOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSource<Tuple2<Tuple2<Integer, Integer>, Integer>> ds = env.fromElements(
+ new Tuple2<>(new Tuple2<>(1, 1), 1),
+ new Tuple2<>(new Tuple2<>(2, 2), 2),
+ new Tuple2<>(new Tuple2<>(2, 2), 2)
+ );
+ ds.partitionByRange(0).withOrders(Order.ASCENDING, Order.DESCENDING);
+ }
+
+ @Test
+ public void testRangePartitionBySelectorComplexKeyWithOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<NestedPojo> ds = getNestedPojoDataSet(env);
+ ds.partitionByRange(new KeySelector<NestedPojo, CustomPojo>() {
+ @Override
+ public CustomPojo getKey(NestedPojo value) throws Exception {
+ return value.getNested();
+ }
+ }).withOrders(Order.ASCENDING);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testRangePartitionBySelectorComplexKeyWithTooManyOrders() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<NestedPojo> ds = getNestedPojoDataSet(env);
+ ds.partitionByRange(new KeySelector<NestedPojo, CustomPojo>() {
+ @Override
+ public CustomPojo getKey(NestedPojo value) throws Exception {
+ return value.getNested();
+ }
+ }).withOrders(Order.ASCENDING, Order.DESCENDING);
+ }
+
+ @Test
+ public void testRangePartitionCustomPartitionerByFieldId() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionCustom(new Partitioner<Integer>() {
+ @Override
+ public int partition(Integer key, int numPartitions) {
+ return 1;
+ }
+ }, 0);
+ }
+
+ @Test(expected = InvalidProgramException.class)
+ public void testRangePartitionInvalidCustomPartitionerByFieldId() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<Tuple2<Integer, String>> ds = getTupleDataSet(env);
+ ds.partitionCustom(new Partitioner<Integer>() {
+ @Override
+ public int partition(Integer key, int numPartitions) {
+ return 1;
+ }
+ }, 1);
+ }
+
+ @Test
+ public void testRangePartitionCustomPartitionerByFieldName() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionCustom(new Partitioner<Integer>() {
+ @Override
+ public int partition(Integer key, int numPartitions) {
+ return 1;
+ }
+ }, "number");
+ }
+
+ @Test(expected = InvalidProgramException.class)
+ public void testRangePartitionInvalidCustomPartitionerByFieldName() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionCustom(new Partitioner<Integer>() {
+ @Override
+ public int partition(Integer key, int numPartitions) {
+ return 1;
+ }
+ }, "name");
+ }
+
+ @Test
+ public void testRangePartitionCustomPartitionerByKeySelector() throws Exception {
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ final DataSet<CustomPojo> ds = getPojoDataSet(env);
+ ds.partitionCustom(new Partitioner<Integer>() {
+ @Override
+ public int partition(Integer key, int numPartitions) {
+ return 1;
+ }
+ }, new KeySelector<CustomPojo, Integer>() {
+ @Override
+ public Integer getKey(CustomPojo value) throws Exception {
+ return value.getNumber();
+ }
+ });
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java
index 048c311..e666704 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/ProjectionOperatorTest.java
@@ -18,25 +18,29 @@
package org.apache.flink.api.java.operator;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+import org.junit.Assert;
import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#project(int...)}.
+ */
public class ProjectionOperatorTest {
// TUPLE DATA
-
- private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+
+ private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -44,112 +48,112 @@ public class ProjectionOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
// LONG DATA
-
+
private final List<Long> emptyLongData = new ArrayList<Long>();
-
+
@Test
public void testFieldsProjection() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.project(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: too many fields
try {
- tupleDs.project(0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25);
+ tupleDs.project(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25);
Assert.fail();
- } catch(IllegalArgumentException iae) {
+ } catch (IllegalArgumentException iae) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: index out of bounds of input tuple
try {
- tupleDs.project(0,5,2);
+ tupleDs.project(0, 5, 2);
Assert.fail();
- } catch(IndexOutOfBoundsException ioobe) {
+ } catch (IndexOutOfBoundsException ioobe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: not applied to tuple dataset
DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
try {
longDs.project(0);
Assert.fail();
- } catch(UnsupportedOperationException uoe) {
+ } catch (UnsupportedOperationException uoe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
-
+
@Test
public void testProjectionTypes() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.project(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should work: dummy types() here
try {
- tupleDs.project(2,1,4);
- } catch(Exception e) {
+ tupleDs.project(2, 1, 4);
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
-
+
@Test
public void testProjectionWithoutTypes() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
- tupleDs.project(2,0,4);
- } catch(Exception e) {
+ tupleDs.project(2, 0, 4);
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: field index is out of bounds of input tuple
try {
- tupleDs.project(2,-1,4);
+ tupleDs.project(2, -1, 4);
Assert.fail();
- } catch(IndexOutOfBoundsException iob) {
+ } catch (IndexOutOfBoundsException iob) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: field index is out of bounds of input tuple
try {
- tupleDs.project(2,1,4,5,8,9);
+ tupleDs.project(2, 1, 4, 5, 8, 9);
Assert.fail();
- } catch(IndexOutOfBoundsException iob) {
+ } catch (IndexOutOfBoundsException iob) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
-
+
}
[07/12] flink git commit: [FLINK-7250] [build] Remove jdk8 profile
Posted by ch...@apache.org.
[FLINK-7250] [build] Remove jdk8 profile
This closes #4399.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/82c8e18a
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/82c8e18a
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/82c8e18a
Branch: refs/heads/master
Commit: 82c8e18a8f15c4e24da35138698597caaad2e58d
Parents: 0e553e0
Author: zentol <ch...@apache.org>
Authored: Mon Jul 24 13:14:33 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:09 2017 +0200
----------------------------------------------------------------------
.travis.yml | 20 ++++++++++----------
pom.xml | 35 +----------------------------------
2 files changed, 11 insertions(+), 44 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/82c8e18a/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 89365c5..a087f45 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -24,52 +24,52 @@ matrix:
- jdk: "oraclejdk8"
env:
- TEST="core"
- - PROFILE="-Dhadoop.version=2.8.0 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.8.0"
- CACHE_NAME=JDK8_H280_CO
- jdk: "oraclejdk8"
env:
- TEST="libraries"
- - PROFILE="-Dhadoop.version=2.8.0 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.8.0"
- CACHE_NAME=JDK8_H280_L
- jdk: "oraclejdk8"
env:
- TEST="connectors"
- - PROFILE="-Dhadoop.version=2.8.0 -Pjdk8,include-kinesis"
+ - PROFILE="-Dhadoop.version=2.8.0 -Pinclude-kinesis"
- CACHE_NAME=JDK8_H280_CN
- jdk: "oraclejdk8"
env:
- TEST="tests"
- - PROFILE="-Dhadoop.version=2.8.0 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.8.0"
- CACHE_NAME=JDK8_H280_T
- jdk: "oraclejdk8"
env:
- TEST="misc"
- - PROFILE="-Dhadoop.version=2.8.0 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.8.0"
- CACHE_NAME=JDK8_H280_M
- jdk: "openjdk8"
env:
- TEST="core"
- - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10"
- CACHE_NAME=JDK8_H241_CO
- jdk: "openjdk8"
env:
- TEST="libraries"
- - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10"
- CACHE_NAME=JDK8_H241_L
- jdk: "openjdk8"
env:
- TEST="connectors"
- - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10 -Pjdk8,include-kinesis"
+ - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10 -Pinclude-kinesis"
- CACHE_NAME=JDK8_H241_CN
- jdk: "openjdk8"
env:
- TEST="tests"
- - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10"
- CACHE_NAME=JDK8_H241_T
- jdk: "openjdk8"
env:
- TEST="misc"
- - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10 -Pjdk8"
+ - PROFILE="-Dhadoop.version=2.4.1 -Dscala-2.10"
- CACHE_NAME=JDK8_H241_M
git:
http://git-wip-us.apache.org/repos/asf/flink/blob/82c8e18a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3287f61..132f73b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -57,6 +57,7 @@ under the License.
<module>flink-shaded-curator</module>
<module>flink-core</module>
<module>flink-java</module>
+ <module>flink-java8</module>
<module>flink-scala</module>
<module>flink-runtime</module>
<module>flink-runtime-web</module>
@@ -830,40 +831,6 @@ under the License.
</pluginManagement>
</build>
</profile>
- <profile>
- <id>jdk8</id>
- <!-- do not activate automatically to prevent 1.8 target while Java 7 is the supported -->
- <properties>
- <java.version>1.8</java.version>
- </properties>
- <modules>
- <module>flink-java8</module>
- </modules>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-javadoc-plugin</artifactId>
- <version>2.9.1</version><!--$NO-MVN-MAN-VER$-->
- <configuration>
- <quiet>true</quiet>
- </configuration>
- <executions>
- <execution>
- <id>attach-javadocs</id>
- <goals>
- <goal>jar</goal>
- </goals>
- <configuration>
- <additionalparam>-Xdoclint:none</additionalparam>
- <detectOfflineLinks>false</detectOfflineLinks>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
- </profile>
</profiles>
<build>
[03/12] flink git commit: [FLINK-7192] [java] Activate checkstyle
flink-java/test/operator
Posted by ch...@apache.org.
[FLINK-7192] [java] Activate checkstyle flink-java/test/operator
This closes #4335.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ca5d8afe
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ca5d8afe
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ca5d8afe
Branch: refs/heads/master
Commit: ca5d8afee8321e0dff063e8404538b132e979739
Parents: 80468b1
Author: Dawid Wysakowicz <dw...@apache.org>
Authored: Fri Jul 14 10:41:50 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:11:11 2017 +0200
----------------------------------------------------------------------
.../java/operator/AggregateOperatorTest.java | 56 +-
.../api/java/operator/CoGroupOperatorTest.java | 179 ++---
.../api/java/operator/CrossOperatorTest.java | 105 +--
.../flink/api/java/operator/DataSinkTest.java | 15 +-
.../api/java/operator/DistinctOperatorTest.java | 93 +--
.../api/java/operator/FirstNOperatorTest.java | 90 +--
.../operator/FullOuterJoinOperatorTest.java | 12 +-
.../java/operator/GroupCombineOperatorTest.java | 36 +-
.../java/operator/GroupReduceOperatorTest.java | 36 +-
.../flink/api/java/operator/GroupingTest.java | 145 ++--
.../api/java/operator/JoinOperatorTest.java | 447 +++++------
.../operator/LeftOuterJoinOperatorTest.java | 17 +-
.../api/java/operator/MaxByOperatorTest.java | 51 +-
.../api/java/operator/MinByOperatorTest.java | 54 +-
.../flink/api/java/operator/OperatorTest.java | 4 +
.../java/operator/PartitionOperatorTest.java | 746 ++++++++++---------
.../java/operator/ProjectionOperatorTest.java | 100 +--
.../api/java/operator/ReduceOperatorTest.java | 28 +-
.../operator/RightOuterJoinOperatorTest.java | 12 +-
.../api/java/operator/SortPartitionTest.java | 38 +-
20 files changed, 1179 insertions(+), 1085 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java
index d9678eb..7977b68 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/AggregateOperatorTest.java
@@ -18,28 +18,32 @@
package org.apache.flink.api.java.operator;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.aggregation.Aggregations;
import org.apache.flink.api.java.aggregation.UnsupportedAggregationTypeException;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+import org.junit.Assert;
import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#aggregate(Aggregations, int)}.
+ */
public class AggregateOperatorTest {
// TUPLE DATA
-
- private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+
+ private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -47,59 +51,59 @@ public class AggregateOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
// LONG DATA
-
+
private final List<Long> emptyLongData = new ArrayList<Long>();
-
+
@Test
public void testFieldsAggregate() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.aggregate(Aggregations.SUM, 1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: index out of bounds
try {
tupleDs.aggregate(Aggregations.SUM, 10);
Assert.fail();
- } catch(IllegalArgumentException iae) {
+ } catch (IllegalArgumentException iae) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work: not applied to tuple dataset
DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
try {
longDs.aggregate(Aggregations.MIN, 1);
Assert.fail();
- } catch(InvalidProgramException uoe) {
+ } catch (InvalidProgramException uoe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
-
+
@Test
public void testAggregationTypes() {
try {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
-
+
// should work: multiple aggregates
tupleDs.aggregate(Aggregations.SUM, 0).and(Aggregations.MIN, 4);
// should work: nested aggregates
tupleDs.aggregate(Aggregations.MIN, 2).aggregate(Aggregations.SUM, 1);
-
+
// should not work: average on string
try {
tupleDs.aggregate(Aggregations.SUM, 2);
@@ -108,7 +112,7 @@ public class AggregateOperatorTest {
// we're good here
}
}
- catch(Exception e) {
+ catch (Exception e) {
System.err.println(e.getMessage());
e.printStackTrace();
Assert.fail(e.getMessage());
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java
index 90a65e6..d8b5f00 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/CoGroupOperatorTest.java
@@ -32,6 +32,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.util.Collector;
+
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -41,14 +42,17 @@ import java.util.List;
import static org.junit.Assert.assertTrue;
+/**
+ * Tests for {@link DataSet#coGroup(DataSet)}.
+ */
@SuppressWarnings("serial")
public class CoGroupOperatorTest {
// TUPLE DATA
- private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+ private static final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -56,17 +60,17 @@ public class CoGroupOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
private static List<CustomType> customTypeData = new ArrayList<CustomType>();
-
+
@BeforeClass
public static void insertCustomData() {
customTypeData.add(new CustomType());
}
-
- @Test
+
+ @Test
public void testCoGroupKeyFields1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -74,14 +78,14 @@ public class CoGroupOperatorTest {
// should work
try {
ds1.coGroup(ds2).where(0).equalTo(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test(expected = InvalidProgramException.class)
public void testCoGroupKeyFields2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -89,21 +93,21 @@ public class CoGroupOperatorTest {
// should not work, incompatible cogroup key types
ds1.coGroup(ds2).where(0).equalTo(2);
}
-
+
@Test(expected = InvalidProgramException.class)
public void testCoGroupKeyFields3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should not work, incompatible number of cogroup keys
- ds1.coGroup(ds2).where(0,1).equalTo(2);
+ ds1.coGroup(ds2).where(0, 1).equalTo(2);
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testCoGroupKeyFields4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -111,10 +115,10 @@ public class CoGroupOperatorTest {
// should not work, cogroup key out of range
ds1.coGroup(ds2).where(5).equalTo(0);
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testCoGroupKeyFields5() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -122,10 +126,10 @@ public class CoGroupOperatorTest {
// should not work, negative key field position
ds1.coGroup(ds2).where(-1).equalTo(-1);
}
-
+
@Test(expected = InvalidProgramException.class)
public void testCoGroupKeyFields6() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -144,7 +148,7 @@ public class CoGroupOperatorTest {
// should work
try {
ds1.coGroup(ds2).where("myInt").equalTo("myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -253,7 +257,7 @@ public class CoGroupOperatorTest {
ds1.coGroup(ds2).where("*").equalTo("*");
}
-
+
@Test
public void testCoGroupKeyExpressions1Nested() {
@@ -264,7 +268,7 @@ public class CoGroupOperatorTest {
// should work
try {
ds1.coGroup(ds2).where("nested.myInt").equalTo("nested.myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
e.printStackTrace();
Assert.fail();
}
@@ -302,10 +306,10 @@ public class CoGroupOperatorTest {
// should not work, cogroup key non-existent
ds1.coGroup(ds2).where("nested.myNonExistent").equalTo("nested.myInt");
}
-
+
@Test
public void testCoGroupKeySelectors1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomType> ds1 = env.fromCollection(customTypeData);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -315,7 +319,7 @@ public class CoGroupOperatorTest {
ds1.coGroup(ds2)
.where(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -324,32 +328,31 @@ public class CoGroupOperatorTest {
)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
}
}
);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCoGroupKeyMixing1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<CustomType> ds1 = env.fromCollection(customTypeData);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
-
// should work
try {
ds1.coGroup(ds2)
.where(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -357,14 +360,14 @@ public class CoGroupOperatorTest {
}
)
.equalTo(3);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCoGroupKeyMixing2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -375,21 +378,21 @@ public class CoGroupOperatorTest {
.where(3)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
}
}
);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test(expected = InvalidProgramException.class)
public void testCoGroupKeyMixing3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
@@ -399,7 +402,7 @@ public class CoGroupOperatorTest {
.where(2)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -407,20 +410,20 @@ public class CoGroupOperatorTest {
}
);
}
-
+
@Test(expected = InvalidProgramException.class)
public void testCoGroupKeyMixing4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> ds1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<CustomType> ds2 = env.fromCollection(customTypeData);
// should not work, more than one key field position
ds1.coGroup(ds2)
- .where(1,3)
+ .where(1, 3)
.equalTo(
new KeySelector<CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
@@ -436,32 +439,32 @@ public class CoGroupOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
- CoGroupOperator<?,?,?> coGroupOp = tupleDs1.coGroup(tupleDs2)
+ CoGroupOperator<?, ?, ?> coGroupOp = tupleDs1.coGroup(tupleDs2)
.where(new DummyTestKeySelector()).equalTo(new DummyTestKeySelector())
.with(new DummyTestCoGroupFunction1());
SemanticProperties semProps = coGroupOp.getSemanticProperties();
assertTrue(semProps.getForwardingTargetFields(0, 0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,2).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(0,2).contains(4));
- assertTrue(semProps.getForwardingTargetFields(0,3).size() == 2);
- assertTrue(semProps.getForwardingTargetFields(0,3).contains(1));
- assertTrue(semProps.getForwardingTargetFields(0,3).contains(3));
- assertTrue(semProps.getForwardingTargetFields(0,4).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,6).size() == 0);
-
- assertTrue(semProps.getForwardingTargetFields(1,0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,2).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,3).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,4).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(1,4).contains(2));
- assertTrue(semProps.getForwardingTargetFields(1,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,6).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(1,6).contains(0));
+ assertTrue(semProps.getForwardingTargetFields(0, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 2).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(0, 2).contains(4));
+ assertTrue(semProps.getForwardingTargetFields(0, 3).size() == 2);
+ assertTrue(semProps.getForwardingTargetFields(0, 3).contains(1));
+ assertTrue(semProps.getForwardingTargetFields(0, 3).contains(3));
+ assertTrue(semProps.getForwardingTargetFields(0, 4).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 6).size() == 0);
+
+ assertTrue(semProps.getForwardingTargetFields(1, 0).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 2).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 3).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 4).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(1, 4).contains(2));
+ assertTrue(semProps.getForwardingTargetFields(1, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 6).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(1, 6).contains(0));
assertTrue(semProps.getReadFields(0).size() == 3);
assertTrue(semProps.getReadFields(0).contains(2));
@@ -480,7 +483,7 @@ public class CoGroupOperatorTest {
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs1 = env.fromCollection(emptyTupleData, tupleTypeInfo);
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs2 = env.fromCollection(emptyTupleData, tupleTypeInfo);
- CoGroupOperator<?,?,?> coGroupOp = tupleDs1.coGroup(tupleDs2)
+ CoGroupOperator<?, ?, ?> coGroupOp = tupleDs1.coGroup(tupleDs2)
.where(new DummyTestKeySelector()).equalTo(new DummyTestKeySelector())
.with(new DummyTestCoGroupFunction2())
.withForwardedFieldsFirst("2;4->0")
@@ -488,26 +491,26 @@ public class CoGroupOperatorTest {
SemanticProperties semProps = coGroupOp.getSemanticProperties();
- assertTrue(semProps.getForwardingTargetFields(0,0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,2).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,3).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,4).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(0,4).contains(2));
- assertTrue(semProps.getForwardingTargetFields(0,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(0,6).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(0,6).contains(0));
-
- assertTrue(semProps.getForwardingTargetFields(1,0).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,1).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,2).size() == 1);
- assertTrue(semProps.getForwardingTargetFields(1,2).contains(4));
- assertTrue(semProps.getForwardingTargetFields(1,3).size() == 2);
- assertTrue(semProps.getForwardingTargetFields(1,3).contains(1));
- assertTrue(semProps.getForwardingTargetFields(1,3).contains(3));
- assertTrue(semProps.getForwardingTargetFields(1,4).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,5).size() == 0);
- assertTrue(semProps.getForwardingTargetFields(1,6).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 0).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 2).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 3).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 4).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(0, 4).contains(2));
+ assertTrue(semProps.getForwardingTargetFields(0, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(0, 6).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(0, 6).contains(0));
+
+ assertTrue(semProps.getForwardingTargetFields(1, 0).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 1).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 2).size() == 1);
+ assertTrue(semProps.getForwardingTargetFields(1, 2).contains(4));
+ assertTrue(semProps.getForwardingTargetFields(1, 3).size() == 2);
+ assertTrue(semProps.getForwardingTargetFields(1, 3).contains(1));
+ assertTrue(semProps.getForwardingTargetFields(1, 3).contains(3));
+ assertTrue(semProps.getForwardingTargetFields(1, 4).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 5).size() == 0);
+ assertTrue(semProps.getForwardingTargetFields(1, 6).size() == 0);
assertTrue(semProps.getReadFields(0).size() == 3);
assertTrue(semProps.getReadFields(0).contains(2));
@@ -517,7 +520,7 @@ public class CoGroupOperatorTest {
assertTrue(semProps.getReadFields(1) == null);
}
- public static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
+ private static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
@Override
public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, String, Long, Integer> value) throws Exception {
return new Tuple2<Long, Integer>();
@@ -528,7 +531,7 @@ public class CoGroupOperatorTest {
@FunctionAnnotation.ForwardedFieldsSecond("2;4->0")
@FunctionAnnotation.ReadFieldsFirst("0;2;4")
@FunctionAnnotation.ReadFieldsSecond("1;3")
- public static class DummyTestCoGroupFunction1
+ private static class DummyTestCoGroupFunction1
implements CoGroupFunction<Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>> {
@@ -541,7 +544,7 @@ public class CoGroupOperatorTest {
}
@FunctionAnnotation.ReadFieldsFirst("0;1;2")
- public static class DummyTestCoGroupFunction2
+ private static class DummyTestCoGroupFunction2
implements CoGroupFunction<Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>,
Tuple5<Integer, Long, String, Long, Integer>> {
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
index 59d2d61..22b03fc 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -31,6 +32,9 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
+/**
+ * Tests for {@link DataSet#cross(DataSet)}.
+ */
public class CrossOperatorTest {
// TUPLE DATA
@@ -64,11 +68,11 @@ public class CrossOperatorTest {
try {
ds1.cross(ds2)
.projectFirst(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection21() {
@@ -80,7 +84,7 @@ public class CrossOperatorTest {
try {
ds1.cross(ds2)
.projectFirst(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -95,12 +99,12 @@ public class CrossOperatorTest {
// should work
try {
ds1.cross(ds2)
- .projectFirst(0,3);
- } catch(Exception e) {
+ .projectFirst(0, 3);
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection22() {
@@ -111,8 +115,8 @@ public class CrossOperatorTest {
// should work
try {
ds1.cross(ds2)
- .projectFirst(0,3);
- } catch(Exception e) {
+ .projectFirst(0, 3);
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -129,11 +133,11 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectFirst(0)
.projectSecond(3);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection23() {
@@ -146,7 +150,7 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectFirst(0)
.projectSecond(3);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -161,15 +165,15 @@ public class CrossOperatorTest {
// should work
try {
ds1.cross(ds2)
- .projectFirst(0,2)
- .projectSecond(1,4)
+ .projectFirst(0, 2)
+ .projectSecond(1, 4)
.projectFirst(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection24() {
@@ -180,10 +184,10 @@ public class CrossOperatorTest {
// should work
try {
ds1.cross(ds2)
- .projectFirst(0,2)
- .projectSecond(1,4)
+ .projectFirst(0, 2)
+ .projectSecond(1, 4)
.projectFirst(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
@@ -199,14 +203,14 @@ public class CrossOperatorTest {
// should work
try {
ds1.cross(ds2)
- .projectSecond(0,2)
- .projectFirst(1,4)
+ .projectSecond(0, 2)
+ .projectFirst(1, 4)
.projectFirst(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection25() {
@@ -217,10 +221,10 @@ public class CrossOperatorTest {
// should work
try {
ds1.cross(ds2)
- .projectSecond(0,2)
- .projectFirst(1,4)
+ .projectSecond(0, 2)
+ .projectFirst(1, 4)
.projectFirst(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -237,11 +241,11 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectFirst()
.projectSecond();
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection26() {
@@ -254,7 +258,7 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectFirst()
.projectSecond();
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -270,12 +274,12 @@ public class CrossOperatorTest {
try {
ds1.cross(ds2)
.projectSecond()
- .projectFirst(1,4);
- } catch(Exception e) {
+ .projectFirst(1, 4);
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testCrossProjection27() {
@@ -287,13 +291,13 @@ public class CrossOperatorTest {
try {
ds1.cross(ds2)
.projectSecond()
- .projectFirst(1,4);
- } catch(Exception e) {
+ .projectFirst(1, 4);
+ } catch (Exception e) {
Assert.fail();
}
}
- @Test(expected=IndexOutOfBoundsException.class)
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection8() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -304,8 +308,8 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectFirst(5);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection28() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -317,7 +321,7 @@ public class CrossOperatorTest {
.projectFirst(5);
}
- @Test(expected=IndexOutOfBoundsException.class)
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection9() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -329,7 +333,7 @@ public class CrossOperatorTest {
.projectSecond(5);
}
- @Test(expected=IndexOutOfBoundsException.class)
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection29() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -351,8 +355,8 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectFirst(2);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection30() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -375,7 +379,7 @@ public class CrossOperatorTest {
.projectSecond(2);
}
- @Test(expected=IndexOutOfBoundsException.class)
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection31() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -386,7 +390,7 @@ public class CrossOperatorTest {
ds1.cross(ds2)
.projectSecond(-1);
}
-
+
public void testCrossProjection12() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -398,8 +402,8 @@ public class CrossOperatorTest {
.projectSecond(2)
.projectFirst(1);
}
-
- @Test(expected=IndexOutOfBoundsException.class)
+
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection32() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -412,7 +416,7 @@ public class CrossOperatorTest {
.projectFirst(-1);
}
- @Test(expected=IndexOutOfBoundsException.class)
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection13() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -425,7 +429,7 @@ public class CrossOperatorTest {
.projectFirst(5);
}
- @Test(expected=IndexOutOfBoundsException.class)
+ @Test(expected = IndexOutOfBoundsException.class)
public void testCrossProjection14() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -437,12 +441,12 @@ public class CrossOperatorTest {
.projectFirst(0)
.projectSecond(5);
}
-
+
/*
* ####################################################################
*/
- public static class CustomType implements Serializable {
+ private static class CustomType implements Serializable {
private static final long serialVersionUID = 1L;
@@ -450,7 +454,8 @@ public class CrossOperatorTest {
public long myLong;
public String myString;
- public CustomType() {}
+ public CustomType() {
+ }
public CustomType(int i, long l, String s) {
myInt = i;
@@ -460,7 +465,7 @@ public class CrossOperatorTest {
@Override
public String toString() {
- return myInt+","+myLong+","+myString;
+ return myInt + "," + myLong + "," + myString;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java
index 0493583..ab8398f 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/DataSinkTest.java
@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.flink.api.java.operator;
import org.apache.flink.api.common.InvalidProgramException;
@@ -25,6 +26,7 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -33,6 +35,9 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
+/**
+ * Tests for {@link DataSet#writeAsText(String)}.
+ */
public class DataSinkTest {
// TUPLE DATA
@@ -48,7 +53,7 @@ public class DataSinkTest {
@Before
public void fillPojoData() {
- if(pojoData.isEmpty()) {
+ if (pojoData.isEmpty()) {
pojoData.add(new CustomType());
}
}
@@ -186,7 +191,7 @@ public class DataSinkTest {
final ExecutionEnvironment env = ExecutionEnvironment
.getExecutionEnvironment();
DataSet<Long> longDs = env
- .generateSequence(0,2);
+ .generateSequence(0, 2);
// should work
try {
@@ -203,7 +208,7 @@ public class DataSinkTest {
final ExecutionEnvironment env = ExecutionEnvironment
.getExecutionEnvironment();
DataSet<Long> longDs = env
- .generateSequence(0,2);
+ .generateSequence(0, 2);
// must not work
longDs.writeAsText("/tmp/willNotHappen")
@@ -216,7 +221,7 @@ public class DataSinkTest {
final ExecutionEnvironment env = ExecutionEnvironment
.getExecutionEnvironment();
DataSet<Long> longDs = env
- .generateSequence(0,2);
+ .generateSequence(0, 2);
// must not work
longDs.writeAsText("/tmp/willNotHappen")
@@ -229,7 +234,7 @@ public class DataSinkTest {
final ExecutionEnvironment env = ExecutionEnvironment
.getExecutionEnvironment();
DataSet<Long> longDs = env
- .generateSequence(0,2);
+ .generateSequence(0, 2);
// must not work
longDs.writeAsText("/tmp/willNotHappen")
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
index 0bbeeb2..b768389 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
@@ -18,27 +18,31 @@
package org.apache.flink.api.java.operator;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
+import org.junit.Assert;
import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#distinct()}.
+ */
public class DistinctOperatorTest {
// TUPLE DATA
- private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+ private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -46,52 +50,52 @@ public class DistinctOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
// LONG DATA
private final List<Long> emptyLongData = new ArrayList<Long>();
-
+
private final List<CustomType> customTypeData = new ArrayList<CustomType>();
-
- @Test
+
+ @Test
public void testDistinctByKeyFields1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.distinct(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
- @Test(expected = InvalidProgramException.class)
+
+ @Test(expected = InvalidProgramException.class)
public void testDistinctByKeyFields2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
// should not work: distinct on basic type
longDs.distinct(0);
}
-
- @Test(expected = InvalidProgramException.class)
+
+ @Test(expected = InvalidProgramException.class)
public void testDistinctByKeyFields3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should not work: distinct on custom type
customDs.distinct(0);
-
+
}
-
+
@Test
public void testDistinctByKeyFields4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -101,17 +105,17 @@ public class DistinctOperatorTest {
@Test
public void testDistinctByKeyFields5() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should work
customDs.distinct();
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testDistinctByKeyFields6() {
@@ -134,29 +138,29 @@ public class DistinctOperatorTest {
Assert.fail();
}
}
-
+
@Test
@SuppressWarnings("serial")
public void testDistinctByKeySelector1() {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
this.customTypeData.add(new CustomType());
-
+
try {
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should work
customDs.distinct(
new KeySelector<DistinctOperatorTest.CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
}
});
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
@Test
@@ -166,7 +170,7 @@ public class DistinctOperatorTest {
DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
// should work
longDs.distinct();
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -206,14 +210,17 @@ public class DistinctOperatorTest {
public List<Integer> myInts;
}
+ /**
+ * Custom data type, for testing purposes.
+ */
public static class CustomType implements Serializable {
-
+
private static final long serialVersionUID = 1L;
-
+
public int myInt;
public long myLong;
public String myString;
-
+
public CustomType() {}
public CustomType(int i, long l, String s) {
@@ -221,11 +228,11 @@ public class DistinctOperatorTest {
myLong = l;
myString = s;
}
-
+
@Override
public String toString() {
- return myInt+","+myLong+","+myString;
+ return myInt + "," + myLong + "," + myString;
}
}
-
+
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java
index aaf744c..b02b318 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/FirstNOperatorTest.java
@@ -18,9 +18,6 @@
package org.apache.flink.api.java.operator;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.operators.Order;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -28,17 +25,24 @@ import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Assert;
import org.junit.Test;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#first(int)}.
+ */
public class FirstNOperatorTest {
// TUPLE DATA
-
- private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+
+ private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -46,127 +50,127 @@ public class FirstNOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
@Test
public void testUngroupedFirstN() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.first(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should work
try {
tupleDs.first(10);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work n == 0
try {
tupleDs.first(0);
Assert.fail();
- } catch(InvalidProgramException ipe) {
+ } catch (InvalidProgramException ipe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work n == -1
try {
tupleDs.first(-1);
Assert.fail();
- } catch(InvalidProgramException ipe) {
+ } catch (InvalidProgramException ipe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
}
-
+
@Test
public void testGroupedFirstN() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.groupBy(2).first(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should work
try {
- tupleDs.groupBy(1,3).first(10);
- } catch(Exception e) {
+ tupleDs.groupBy(1, 3).first(10);
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work n == 0
try {
tupleDs.groupBy(0).first(0);
Assert.fail();
- } catch(InvalidProgramException ipe) {
+ } catch (InvalidProgramException ipe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work n == -1
try {
tupleDs.groupBy(2).first(-1);
Assert.fail();
- } catch(InvalidProgramException ipe) {
+ } catch (InvalidProgramException ipe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
public void testGroupedSortedFirstN() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.groupBy(2).sortGroup(4, Order.ASCENDING).first(1);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should work
try {
- tupleDs.groupBy(1,3).sortGroup(4, Order.ASCENDING).first(10);
- } catch(Exception e) {
+ tupleDs.groupBy(1, 3).sortGroup(4, Order.ASCENDING).first(10);
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work n == 0
try {
tupleDs.groupBy(0).sortGroup(4, Order.ASCENDING).first(0);
Assert.fail();
- } catch(InvalidProgramException ipe) {
+ } catch (InvalidProgramException ipe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
-
+
// should not work n == -1
try {
tupleDs.groupBy(2).sortGroup(4, Order.ASCENDING).first(-1);
Assert.fail();
- } catch(InvalidProgramException ipe) {
+ } catch (InvalidProgramException ipe) {
// we're good here
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
index 9f2aa41..d52f59f 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/FullOuterJoinOperatorTest.java
@@ -28,11 +28,15 @@ import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+
import org.junit.Test;
import java.util.ArrayList;
import java.util.List;
+/**
+ * Tests for {@link DataSet#fullOuterJoin(DataSet)}.
+ */
public class FullOuterJoinOperatorTest {
// TUPLE DATA
@@ -201,7 +205,6 @@ public class FullOuterJoinOperatorTest {
this.testFullOuterStrategies(JoinHint.BROADCAST_HASH_FIRST);
}
-
private void testFullOuterStrategies(JoinHint hint) {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
@@ -214,13 +217,12 @@ public class FullOuterJoinOperatorTest {
.with(new DummyJoin());
}
-
/*
* ####################################################################
*/
@SuppressWarnings("serial")
- public static class DummyJoin implements
+ private static class DummyJoin implements
JoinFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>, Long> {
@Override
@@ -230,7 +232,7 @@ public class FullOuterJoinOperatorTest {
}
@SuppressWarnings("serial")
- public static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
+ private static class IntKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Integer> {
@Override
public Integer getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
@@ -239,7 +241,7 @@ public class FullOuterJoinOperatorTest {
}
@SuppressWarnings("serial")
- public static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
+ private static class LongKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Long> {
@Override
public Long getKey(Tuple5<Integer, Long, String, Long, Integer> v) throws Exception {
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java
index 4870d29..336219b 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupCombineOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.util.Collector;
+
import org.junit.Test;
import java.util.ArrayList;
@@ -38,13 +39,16 @@ import java.util.List;
import static org.junit.Assert.assertTrue;
+/**
+ * Tests for {@link DataSet#combineGroup(GroupCombineFunction)}.
+ */
@SuppressWarnings("serial")
public class GroupCombineOperatorTest {
private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -52,14 +56,14 @@ public class GroupCombineOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
@Test
public void testSemanticPropsWithKeySelector1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.combineGroup(new DummyGroupCombineFunction1());
@@ -95,7 +99,7 @@ public class GroupCombineOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.sortGroup(new DummyTestKeySelector(), Order.ASCENDING)
.combineGroup(new DummyGroupCombineFunction1());
@@ -134,7 +138,7 @@ public class GroupCombineOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.combineGroup(new DummyGroupCombineFunction2())
.withForwardedFields("0->4;1;1->3;2");
@@ -171,7 +175,7 @@ public class GroupCombineOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.sortGroup(new DummyTestKeySelector(), Order.ASCENDING)
.combineGroup(new DummyGroupCombineFunction2())
@@ -211,7 +215,7 @@ public class GroupCombineOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.combineGroup(new DummyGroupCombineFunction3())
.withForwardedFields("4->0;3;3->1;2");
@@ -245,7 +249,7 @@ public class GroupCombineOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.sortGroup(new DummyTestKeySelector(), Order.ASCENDING)
.combineGroup(new DummyGroupCombineFunction3())
@@ -282,7 +286,7 @@ public class GroupCombineOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> combineOp =
+ GroupCombineOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> combineOp =
tupleDs.groupBy(new DummyTestKeySelector())
.combineGroup(new DummyGroupCombineFunction4());
@@ -308,7 +312,7 @@ public class GroupCombineOperatorTest {
assertTrue(semProps.getReadFields(0) == null);
}
- public static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
+ private static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
@Override
public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, String, Long, Integer> value) throws Exception {
return new Tuple2<Long, Integer>();
@@ -317,27 +321,27 @@ public class GroupCombineOperatorTest {
@FunctionAnnotation.ForwardedFields("0->4;1;1->3;2")
@FunctionAnnotation.ReadFields("0;3;4")
- public static class DummyGroupCombineFunction1 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupCombineFunction1 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void combine(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
}
@FunctionAnnotation.ReadFields("0;3;4")
- public static class DummyGroupCombineFunction2 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupCombineFunction2 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void combine(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
}
- public static class DummyGroupCombineFunction3 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupCombineFunction3 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void combine(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
}
@FunctionAnnotation.NonForwardedFields("2;4")
- public static class DummyGroupCombineFunction4 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupCombineFunction4 implements GroupCombineFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void combine(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java
index 0bfe566..62380eb 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupReduceOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.util.Collector;
+
import org.junit.Test;
import java.util.ArrayList;
@@ -38,13 +39,16 @@ import java.util.List;
import static org.junit.Assert.assertTrue;
+/**
+ * Tests for {@link DataSet#reduceGroup(GroupReduceFunction)}.
+ */
@SuppressWarnings("serial")
public class GroupReduceOperatorTest {
private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -52,14 +56,14 @@ public class GroupReduceOperatorTest {
BasicTypeInfo.LONG_TYPE_INFO,
BasicTypeInfo.INT_TYPE_INFO
);
-
+
@Test
public void testSemanticPropsWithKeySelector1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.reduceGroup(new DummyGroupReduceFunction1());
@@ -95,7 +99,7 @@ public class GroupReduceOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.sortGroup(new DummyTestKeySelector(), Order.ASCENDING)
.reduceGroup(new DummyGroupReduceFunction1());
@@ -134,7 +138,7 @@ public class GroupReduceOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.reduceGroup(new DummyGroupReduceFunction2())
.withForwardedFields("0->4;1;1->3;2");
@@ -171,7 +175,7 @@ public class GroupReduceOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.sortGroup(new DummyTestKeySelector(), Order.ASCENDING)
.reduceGroup(new DummyGroupReduceFunction2())
@@ -211,7 +215,7 @@ public class GroupReduceOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.reduceGroup(new DummyGroupReduceFunction3())
.withForwardedFields("4->0;3;3->1;2");
@@ -245,7 +249,7 @@ public class GroupReduceOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.sortGroup(new DummyTestKeySelector(), Order.ASCENDING)
.reduceGroup(new DummyGroupReduceFunction3())
@@ -282,7 +286,7 @@ public class GroupReduceOperatorTest {
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
- GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>,Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
+ GroupReduceOperator<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> reduceOp =
tupleDs.groupBy(new DummyTestKeySelector())
.reduceGroup(new DummyGroupReduceFunction4());
@@ -308,7 +312,7 @@ public class GroupReduceOperatorTest {
assertTrue(semProps.getReadFields(0) == null);
}
- public static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
+ private static class DummyTestKeySelector implements KeySelector<Tuple5<Integer, Long, String, Long, Integer>, Tuple2<Long, Integer>> {
@Override
public Tuple2<Long, Integer> getKey(Tuple5<Integer, Long, String, Long, Integer> value) throws Exception {
return new Tuple2<Long, Integer>();
@@ -317,27 +321,27 @@ public class GroupReduceOperatorTest {
@FunctionAnnotation.ForwardedFields("0->4;1;1->3;2")
@FunctionAnnotation.ReadFields("0;3;4")
- public static class DummyGroupReduceFunction1 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupReduceFunction1 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void reduce(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
}
@FunctionAnnotation.ReadFields("0;3;4")
- public static class DummyGroupReduceFunction2 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupReduceFunction2 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void reduce(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
}
- public static class DummyGroupReduceFunction3 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupReduceFunction3 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void reduce(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
}
@FunctionAnnotation.NonForwardedFields("2;4")
- public static class DummyGroupReduceFunction4 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
+ private static class DummyGroupReduceFunction4 implements GroupReduceFunction<Tuple5<Integer, Long, String, Long, Integer>, Tuple5<Integer, Long, String, Long, Integer>> {
@Override
public void reduce(Iterable<Tuple5<Integer, Long, String, Long, Integer>> values, Collector<Tuple5<Integer, Long, String, Long, Integer>> out) throws Exception {
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ca5d8afe/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
index 18b17b5..0117cf1 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
@@ -18,10 +18,6 @@
package org.apache.flink.api.java.operator;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.operators.Order;
import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
@@ -34,16 +30,24 @@ import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
+
import org.junit.Assert;
import org.junit.Test;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests for {@link DataSet#groupBy(int...)}.
+ */
public class GroupingTest {
// TUPLE DATA
- private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
+ private final List<Tuple5<Integer, Long, String, Long, Integer>> emptyTupleData =
new ArrayList<Tuple5<Integer, Long, String, Long, Integer>>();
-
- private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
+
+ private final TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>> tupleTypeInfo = new
TupleTypeInfo<Tuple5<Integer, Long, String, Long, Integer>>(
BasicTypeInfo.INT_TYPE_INFO,
BasicTypeInfo.LONG_TYPE_INFO,
@@ -62,64 +66,64 @@ public class GroupingTest {
// LONG DATA
private final List<Long> emptyLongData = new ArrayList<Long>();
-
+
private final List<CustomType> customTypeData = new ArrayList<CustomType>();
private final List<Tuple4<Integer, Long, CustomType, Long[]>> tupleWithCustomData =
new ArrayList<Tuple4<Integer, Long, CustomType, Long[]>>();
-
+
private final List<Tuple2<byte[], byte[]>> byteArrayData = new ArrayList<Tuple2<byte[], byte[]>>();
- @Test
+ @Test
public void testGroupByKeyFields1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.groupBy(0);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
- @Test(expected = InvalidProgramException.class)
+
+ @Test(expected = InvalidProgramException.class)
public void testGroupByKeyFields2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
// should not work: groups on basic type
longDs.groupBy(0);
}
-
- @Test(expected = InvalidProgramException.class)
+
+ @Test(expected = InvalidProgramException.class)
public void testGroupByKeyFields3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
+
this.customTypeData.add(new CustomType());
-
+
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should not work: groups on custom type
customDs.groupBy(0);
-
+
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testGroupByKeyFields4() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should not work, key out of tuple bounds
tupleDs.groupBy(5);
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testGroupByKeyFields5() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
@@ -148,7 +152,7 @@ public class GroupingTest {
// should work
try {
ds.groupBy("myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -198,7 +202,7 @@ public class GroupingTest {
// should work
try {
ds.groupBy("nested.myInt");
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -212,48 +216,48 @@ public class GroupingTest {
// should not work, key out of tuple bounds
ds.groupBy("nested.myNonExistent");
}
-
+
@Test
@SuppressWarnings("serial")
public void testGroupByKeySelector1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
this.customTypeData.add(new CustomType());
-
+
try {
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should work
customDs.groupBy(
new KeySelector<GroupingTest.CustomType, Long>() {
-
+
@Override
public Long getKey(CustomType value) {
return value.myLong;
}
});
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test
@SuppressWarnings("serial")
public void testGroupByKeySelector2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
this.customTypeData.add(new CustomType());
-
+
try {
DataSet<CustomType> customDs = env.fromCollection(customTypeData);
// should work
customDs.groupBy(
new KeySelector<GroupingTest.CustomType, Tuple2<Integer, Long>>() {
@Override
- public Tuple2<Integer,Long> getKey(CustomType value) {
+ public Tuple2<Integer, Long> getKey(CustomType value) {
return new Tuple2<Integer, Long>(value.myInt, value.myLong);
}
});
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -261,7 +265,7 @@ public class GroupingTest {
@Test
@SuppressWarnings("serial")
public void testGroupByKeySelector3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
this.customTypeData.add(new CustomType());
@@ -275,7 +279,7 @@ public class GroupingTest {
return value;
}
});
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -297,7 +301,7 @@ public class GroupingTest {
return new Tuple2<Integer, CustomType>(value.myInt, value);
}
});
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -319,38 +323,38 @@ public class GroupingTest {
}
});
}
-
+
@Test
public void testGroupSortKeyFields1() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.groupBy(0).sortGroup(0, Order.ASCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
-
+
@Test(expected = IndexOutOfBoundsException.class)
public void testGroupSortKeyFields2() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should not work, field index out of bounds
tupleDs.groupBy(0).sortGroup(5, Order.ASCENDING);
-
+
}
-
+
@Test(expected = InvalidProgramException.class)
public void testGroupSortKeyFields3() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Long> longDs = env.fromCollection(emptyLongData, BasicTypeInfo.LONG_TYPE_INFO);
-
+
// should not work: sorted groups on groupings by key selectors
longDs.groupBy(new KeySelector<Long, Long>() {
private static final long serialVersionUID = 1L;
@@ -359,9 +363,9 @@ public class GroupingTest {
public Long getKey(Long value) {
return value;
}
-
+
}).sortGroup(0, Order.ASCENDING);
-
+
}
@Test(expected = InvalidProgramException.class)
@@ -387,17 +391,17 @@ public class GroupingTest {
tupleDs.groupBy(0)
.sortGroup(3, Order.ASCENDING);
}
-
+
@Test
public void testChainedGroupSortKeyFields() {
-
+
final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple5<Integer, Long, String, Long, Integer>> tupleDs = env.fromCollection(emptyTupleData, tupleTypeInfo);
// should work
try {
tupleDs.groupBy(0).sortGroup(0, Order.ASCENDING).sortGroup(2, Order.DESCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -412,7 +416,7 @@ public class GroupingTest {
// should work
try {
tupleDs.groupBy("f0").sortGroup("f1", Order.ASCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -427,7 +431,7 @@ public class GroupingTest {
// should work
try {
tupleDs.groupBy("f0").sortGroup("f2.myString", Order.ASCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -444,7 +448,7 @@ public class GroupingTest {
tupleDs.groupBy("f0")
.sortGroup("f2.myString", Order.ASCENDING)
.sortGroup("f1", Order.DESCENDING);
- } catch(Exception e) {
+ } catch (Exception e) {
Assert.fail();
}
}
@@ -496,7 +500,7 @@ public class GroupingTest {
// should not work
tupleDs.groupBy(
- new KeySelector<Tuple4<Integer,Long,CustomType,Long[]>, Long>() {
+ new KeySelector<Tuple4<Integer, Long, CustomType, Long[]>, Long>() {
@Override
public Long getKey(Tuple4<Integer, Long, CustomType, Long[]> value) throws Exception {
return value.f1;
@@ -521,7 +525,7 @@ public class GroupingTest {
// should not work
tupleDs.groupBy(
- new KeySelector<Tuple4<Integer,Long,CustomType,Long[]>, Long>() {
+ new KeySelector<Tuple4<Integer, Long, CustomType, Long[]>, Long>() {
@Override
public Long getKey(Tuple4<Integer, Long, CustomType, Long[]> value) throws Exception {
return value.f1;
@@ -546,7 +550,7 @@ public class GroupingTest {
// should not work
tupleDs.groupBy(
- new KeySelector<Tuple4<Integer,Long,CustomType,Long[]>, Long>() {
+ new KeySelector<Tuple4<Integer, Long, CustomType, Long[]>, Long>() {
@Override
public Long getKey(Tuple4<Integer, Long, CustomType, Long[]> value) throws Exception {
return value.f1;
@@ -593,19 +597,25 @@ public class GroupingTest {
dataSet.groupBy("*");
}
-
+ /**
+ * Custom data type, for testing purposes.
+ */
public static class CustomType implements Serializable {
-
+
+ /**
+ * Custom nested data type, for testing purposes.
+ */
public static class Nest {
public int myInt;
}
+
private static final long serialVersionUID = 1L;
-
+
public int myInt;
public long myLong;
public String myString;
public Nest nested;
-
+
public CustomType() {}
public CustomType(int i, long l, String s) {
@@ -613,13 +623,16 @@ public class GroupingTest {
myLong = l;
myString = s;
}
-
+
@Override
public String toString() {
- return myInt+","+myLong+","+myString;
+ return myInt + "," + myLong + "," + myString;
}
}
+ /**
+ * Custom non-nested data type, for testing purposes.
+ */
public static class CustomType2 implements Serializable {
public int myInt;
[11/12] flink git commit: [hotfix] [tests] fix
NetworkStackThroughputITCase not working anymore
Posted by ch...@apache.org.
[hotfix] [tests] fix NetworkStackThroughputITCase not working anymore
Invokable classes need to stay public in order to be successfully executed by
the TM.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3407a449
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3407a449
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3407a449
Branch: refs/heads/master
Commit: 3407a44922289dc4ddb3ff87687b8766affeaad9
Parents: 738dbbe
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Fri Jul 28 14:18:09 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:09 2017 +0200
----------------------------------------------------------------------
.../runtime/NetworkStackThroughputITCase.java | 31 +++++++++++++++++---
1 file changed, 27 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/3407a449/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
index ba3499f..92bf6d6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java
@@ -163,7 +163,14 @@ public class NetworkStackThroughputITCase extends TestLogger {
// ------------------------------------------------------------------------
- private static class SpeedTestProducer extends AbstractInvokable {
+ /**
+ * Invokable that produces records and allows slowdown via {@link #IS_SLOW_EVERY_NUM_RECORDS}
+ * and {@link #IS_SLOW_SENDER_CONFIG_KEY} and creates records of different data sizes via {@link
+ * #DATA_VOLUME_GB_CONFIG_KEY}.
+ *
+ * <p>NOTE: needs to be <tt>public</tt> so that a task can be run with this!
+ */
+ public static class SpeedTestProducer extends AbstractInvokable {
@Override
public void invoke() throws Exception {
@@ -198,7 +205,12 @@ public class NetworkStackThroughputITCase extends TestLogger {
}
}
- private static class SpeedTestForwarder extends AbstractInvokable {
+ /**
+ * Invokable that forwards incoming records.
+ *
+ * <p>NOTE: needs to be <tt>public</tt> so that a task can be run with this!
+ */
+ public static class SpeedTestForwarder extends AbstractInvokable {
@Override
public void invoke() throws Exception {
@@ -222,7 +234,13 @@ public class NetworkStackThroughputITCase extends TestLogger {
}
}
- private static class SpeedTestConsumer extends AbstractInvokable {
+ /**
+ * Invokable that consumes incoming records and allows slowdown via {@link
+ * #IS_SLOW_EVERY_NUM_RECORDS}.
+ *
+ * <p>NOTE: needs to be <tt>public</tt> so that a task can be run with this!
+ */
+ public static class SpeedTestConsumer extends AbstractInvokable {
@Override
public void invoke() throws Exception {
@@ -247,7 +265,12 @@ public class NetworkStackThroughputITCase extends TestLogger {
}
}
- private static class SpeedTestRecord implements IOReadableWritable {
+ /**
+ * Record type for the speed test.
+ *
+ * <p>NOTE: needs to be <tt>public</tt> to allow deserialization!
+ */
+ public static class SpeedTestRecord implements IOReadableWritable {
private static final int RECORD_SIZE = 128;
[06/12] flink git commit: [FLINK-7188] [java] Activate checkstyle
flink-java/summarize
Posted by ch...@apache.org.
[FLINK-7188] [java] Activate checkstyle flink-java/summarize
This closes #4338.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8bd537fd
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8bd537fd
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8bd537fd
Branch: refs/heads/master
Commit: 8bd537fda257a07dfd15bd4e9adffb7519b516ad
Parents: 3407a44
Author: Dawid Wysakowicz <dw...@apache.org>
Authored: Fri Jul 14 10:33:11 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:09 2017 +0200
----------------------------------------------------------------------
.../java/summarize/BooleanColumnSummary.java | 4 +-
.../flink/api/java/summarize/ColumnSummary.java | 12 ++---
.../java/summarize/NumericColumnSummary.java | 16 +++----
.../api/java/summarize/ObjectColumnSummary.java | 2 +-
.../api/java/summarize/StringColumnSummary.java | 8 ++--
.../java/summarize/aggregation/Aggregator.java | 12 ++---
.../aggregation/BooleanSummaryAggregator.java | 13 +++---
.../summarize/aggregation/CompensatedSum.java | 16 +++----
.../aggregation/DoubleSummaryAggregator.java | 25 +++++++----
.../aggregation/FloatSummaryAggregator.java | 21 ++++++---
.../aggregation/IntegerSummaryAggregator.java | 21 ++++++---
.../aggregation/LongSummaryAggregator.java | 21 ++++++---
.../aggregation/NumericSummaryAggregator.java | 28 ++++++------
.../aggregation/ObjectSummaryAggregator.java | 5 ++-
.../aggregation/ShortSummaryAggregator.java | 25 +++++++----
.../aggregation/StringSummaryAggregator.java | 9 ++--
.../aggregation/SummaryAggregatorFactory.java | 36 +++++++--------
.../aggregation/TupleSummaryAggregator.java | 8 ++--
.../aggregation/ValueSummaryAggregator.java | 46 ++++++++++++++------
.../aggregation/AggregateCombineHarness.java | 12 ++---
.../BooleanSummaryAggregatorTest.java | 7 ++-
.../BooleanValueSummaryAggregatorTest.java | 8 +++-
.../aggregation/CompensatedSumTest.java | 9 ++--
.../DoubleSummaryAggregatorTest.java | 16 ++++---
.../DoubleValueSummaryAggregatorTest.java | 10 +++--
.../aggregation/FloatSummaryAggregatorTest.java | 17 +++++---
.../FloatValueSummaryAggregatorTest.java | 10 +++--
.../IntegerSummaryAggregatorTest.java | 32 ++++++++------
.../IntegerValueSummaryAggregatorTest.java | 30 +++++++------
.../aggregation/LongSummaryAggregatorTest.java | 28 +++++++-----
.../LongValueSummaryAggregatorTest.java | 30 +++++++------
.../aggregation/ShortSummaryAggregatorTest.java | 39 +++++++++--------
.../ShortValueSummaryAggregatorTest.java | 32 ++++++++------
.../StringSummaryAggregatorTest.java | 9 ++--
.../StringValueSummaryAggregatorTest.java | 10 +++--
.../SummaryAggregatorFactoryTest.java | 13 +++++-
tools/maven/suppressions-java.xml | 8 ----
37 files changed, 388 insertions(+), 260 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/BooleanColumnSummary.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/BooleanColumnSummary.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/BooleanColumnSummary.java
index 944808f..8bd219b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/BooleanColumnSummary.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/BooleanColumnSummary.java
@@ -21,7 +21,7 @@ package org.apache.flink.api.java.summarize;
import org.apache.flink.annotation.PublicEvolving;
/**
- * Summary for a column of booleans
+ * Summary for a column of booleans.
*/
@PublicEvolving
public class BooleanColumnSummary extends ColumnSummary {
@@ -45,7 +45,7 @@ public class BooleanColumnSummary extends ColumnSummary {
}
/**
- * The number of non-null values in this column
+ * The number of non-null values in this column.
*/
@Override
public long getNonNullCount() {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/ColumnSummary.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/ColumnSummary.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/ColumnSummary.java
index b73d60c..76f8dd1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/ColumnSummary.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/ColumnSummary.java
@@ -21,37 +21,37 @@ package org.apache.flink.api.java.summarize;
import org.apache.flink.annotation.PublicEvolving;
/**
- * Summary for a column of values
+ * Summary for a column of values.
*/
@PublicEvolving
public abstract class ColumnSummary {
/**
- * The number of all rows in this column including both nulls and non-nulls
+ * The number of all rows in this column including both nulls and non-nulls.
*/
public long getTotalCount() {
return getNullCount() + getNonNullCount();
}
/**
- * The number of non-null values in this column
+ * The number of non-null values in this column.
*/
public abstract long getNonNullCount();
/**
- * The number of null values in this column
+ * The number of null values in this column.
*/
public abstract long getNullCount();
/**
- * True if this column contains any null values
+ * True if this column contains any null values.
*/
public boolean containsNull() {
return getNullCount() > 0L;
}
/**
- * True if this column contains any non-null values
+ * True if this column contains any non-null values.
*/
public boolean containsNonNull() {
return getNonNullCount() > 0L;
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/NumericColumnSummary.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/NumericColumnSummary.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/NumericColumnSummary.java
index fd53426..2ab5b84 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/NumericColumnSummary.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/NumericColumnSummary.java
@@ -23,10 +23,10 @@ import org.apache.flink.annotation.PublicEvolving;
/**
* Generic Column Summary for Numeric Types.
*
- * Some values are considered "missing" where "missing" is defined as null, NaN, or Infinity.
+ * <p>Some values are considered "missing" where "missing" is defined as null, NaN, or Infinity.
* These values are ignored in some calculations like mean, variance, and standardDeviation.
*
- * Uses the Kahan summation algorithm to avoid numeric instability when computing variance.
+ * <p>Uses the Kahan summation algorithm to avoid numeric instability when computing variance.
* The algorithm is described in: "Scalable and Numerically Stable Descriptive Statistics in SystemML",
* Tian et al, International Conference on Data Engineering 2012.
*
@@ -66,7 +66,7 @@ public class NumericColumnSummary<T> extends ColumnSummary implements java.io.Se
/**
* The number of "missing" values where "missing" is defined as null, NaN, or Infinity.
*
- * These values are ignored in some calculations like mean, variance, and standardDeviation.
+ * <p>These values are ignored in some calculations like mean, variance, and standardDeviation.
*/
public long getMissingCount() {
return nullCount + nanCount + infinityCount;
@@ -80,7 +80,7 @@ public class NumericColumnSummary<T> extends ColumnSummary implements java.io.Se
}
/**
- * The number of non-null values in this column
+ * The number of non-null values in this column.
*/
@Override
public long getNonNullCount() {
@@ -95,7 +95,7 @@ public class NumericColumnSummary<T> extends ColumnSummary implements java.io.Se
/**
* Number of values that are NaN.
*
- * (always zero for types like Short, Integer, Long)
+ * <p>(always zero for types like Short, Integer, Long)
*/
public long getNanCount() {
return nanCount;
@@ -104,7 +104,7 @@ public class NumericColumnSummary<T> extends ColumnSummary implements java.io.Se
/**
* Number of values that are positive or negative infinity.
*
- * (always zero for types like Short, Integer, Long)
+ * <p>(always zero for types like Short, Integer, Long)
*/
public long getInfinityCount() {
return infinityCount;
@@ -134,7 +134,7 @@ public class NumericColumnSummary<T> extends ColumnSummary implements java.io.Se
/**
* Variance is a measure of how far a set of numbers are spread out.
*
- * Null, NaN, and Infinite values are ignored in this calculation.
+ * <p>Null, NaN, and Infinite values are ignored in this calculation.
*
* @see <a href="https://en.wikipedia.org/wiki/Variance">Variance</a>
*/
@@ -145,7 +145,7 @@ public class NumericColumnSummary<T> extends ColumnSummary implements java.io.Se
/**
* Standard Deviation is a measure of variation in a set of numbers. It is the square root of the variance.
*
- * Null, NaN, and Infinite values are ignored in this calculation.
+ * <p>Null, NaN, and Infinite values are ignored in this calculation.
*
* @see <a href="https://en.wikipedia.org/wiki/Standard_deviation">Standard Deviation</a>
*/
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/ObjectColumnSummary.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/ObjectColumnSummary.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/ObjectColumnSummary.java
index 42d9ae3..9e5986e 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/ObjectColumnSummary.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/ObjectColumnSummary.java
@@ -35,7 +35,7 @@ public class ObjectColumnSummary extends ColumnSummary {
}
/**
- * The number of non-null values in this column
+ * The number of non-null values in this column.
*/
@Override
public long getNonNullCount() {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/StringColumnSummary.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/StringColumnSummary.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/StringColumnSummary.java
index 98840da..894c60b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/StringColumnSummary.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/StringColumnSummary.java
@@ -21,7 +21,7 @@ package org.apache.flink.api.java.summarize;
import org.apache.flink.annotation.PublicEvolving;
/**
- * Summary for a column of Strings
+ * Summary for a column of Strings.
*/
@PublicEvolving
public class StringColumnSummary extends ColumnSummary {
@@ -53,21 +53,21 @@ public class StringColumnSummary extends ColumnSummary {
}
/**
- * Number of empty strings e.g. java.lang.String.isEmpty()
+ * Number of empty strings e.g. java.lang.String.isEmpty().
*/
public long getEmptyCount() {
return emptyCount;
}
/**
- * Shortest String length
+ * Shortest String length.
*/
public Integer getMinLength() {
return minLength;
}
/**
- * Longest String length
+ * Longest String length.
*/
public Integer getMaxLength() {
return maxLength;
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/Aggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/Aggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/Aggregator.java
index 2ece022..5002a3c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/Aggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/Aggregator.java
@@ -21,24 +21,24 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
/**
- * Generic interface for aggregation
+ * Generic interface for aggregation.
*
* @param <T> the type to be aggregated
* @param <R> the result type of the aggregation
*/
@Internal
-public interface Aggregator<T,R> extends java.io.Serializable {
+public interface Aggregator<T, R> extends java.io.Serializable {
- /** Add a value to the current aggregation */
+ /** Add a value to the current aggregation. */
void aggregate(T value);
/**
* Combine two aggregations of the same type.
*
- * (Implementations will need to do an unchecked cast).
+ * <p>(Implementations will need to do an unchecked cast).
*/
- void combine(Aggregator<T,R> otherSameType);
+ void combine(Aggregator<T, R> otherSameType);
- /** Provide the final result of the aggregation */
+ /** Provide the final result of the aggregation. */
R result();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregator.java
index bc3022e..b3c8901 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregator.java
@@ -21,8 +21,11 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.java.summarize.BooleanColumnSummary;
+/**
+ * Aggregator for {@link Boolean} type.
+ */
@Internal
-public class BooleanSummaryAggregator implements Aggregator<Boolean,BooleanColumnSummary> {
+public class BooleanSummaryAggregator implements Aggregator<Boolean, BooleanColumnSummary> {
private long trueCount = 0L;
private long falseCount = 0L;
@@ -32,11 +35,9 @@ public class BooleanSummaryAggregator implements Aggregator<Boolean,BooleanColum
public void aggregate(Boolean value) {
if (value == null) {
nullCount++;
- }
- else if (value) {
+ } else if (value) {
trueCount++;
- }
- else {
+ } else {
falseCount++;
}
}
@@ -51,6 +52,6 @@ public class BooleanSummaryAggregator implements Aggregator<Boolean,BooleanColum
@Override
public BooleanColumnSummary result() {
- return new BooleanColumnSummary(trueCount,falseCount,nullCount);
+ return new BooleanColumnSummary(trueCount, falseCount, nullCount);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSum.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSum.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSum.java
index 0a90f6d..dac0c11 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSum.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSum.java
@@ -21,9 +21,9 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
/**
- * Used to calculate sums using the Kahan summation algorithm
+ * Used to calculate sums using the Kahan summation algorithm.
*
- * The Kahan summation algorithm (also known as compensated summation) reduces the numerical errors that
+ * <p>The Kahan summation algorithm (also known as compensated summation) reduces the numerical errors that
* occur when adding a sequence of finite precision floating point numbers. Numerical errors arise due to
* truncation and rounding. These errors can lead to numerical instability.
*
@@ -41,7 +41,7 @@ public class CompensatedSum implements java.io.Serializable {
private final double delta;
/**
- * Used to calculate sums using the Kahan summation algorithm
+ * Used to calculate sums using the Kahan summation algorithm.
* @param value the sum
* @param delta correction term
*/
@@ -51,35 +51,35 @@ public class CompensatedSum implements java.io.Serializable {
}
/**
- * The value of the sum
+ * The value of the sum.
*/
public double value() {
return value;
}
/**
- * The correction term
+ * The correction term.
*/
public double delta() {
return delta;
}
/**
- * Increments the Kahan sum by adding a value and a correction term
+ * Increments the Kahan sum by adding a value and a correction term.
*/
public CompensatedSum add(double value, double delta) {
return add(new CompensatedSum(value, delta));
}
/**
- * Increments the Kahan sum by adding a value without a correction term
+ * Increments the Kahan sum by adding a value without a correction term.
*/
public CompensatedSum add(double value) {
return add(new CompensatedSum(value, NO_CORRECTION));
}
/**
- * Increments the Kahan sum by adding two sums, and updating the correction term for reducing numeric errors
+ * Increments the Kahan sum by adding two sums, and updating the correction term for reducing numeric errors.
*/
public CompensatedSum add(CompensatedSum other) {
double correctedSum = other.value() + (delta + other.delta());
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregator.java
index cdf40c5..a5154e5 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregator.java
@@ -18,19 +18,22 @@
package org.apache.flink.api.java.summarize.aggregation;
-import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
-
import org.apache.flink.annotation.Internal;
+import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
+
/**
- * Aggregator that can handle Double types
+ * Aggregator that can handle Double types.
*/
@Internal
public class DoubleSummaryAggregator extends NumericSummaryAggregator<Double> {
// Nested classes are only "public static" for Kryo serialization, otherwise they'd be private
- public static class MinDoubleAggregator implements Aggregator<Double,Double> {
+ /**
+ * Aggregator for min operation.
+ */
+ public static class MinDoubleAggregator implements Aggregator<Double, Double> {
private double min = Double.POSITIVE_INFINITY;
@@ -41,7 +44,7 @@ public class DoubleSummaryAggregator extends NumericSummaryAggregator<Double> {
@Override
public void combine(Aggregator<Double, Double> other) {
- min = Math.min(min,((MinDoubleAggregator)other).min);
+ min = Math.min(min, ((MinDoubleAggregator) other).min);
}
@Override
@@ -50,7 +53,10 @@ public class DoubleSummaryAggregator extends NumericSummaryAggregator<Double> {
}
}
- public static class MaxDoubleAggregator implements Aggregator<Double,Double> {
+ /**
+ * Aggregator for max operation.
+ */
+ public static class MaxDoubleAggregator implements Aggregator<Double, Double> {
private double max = Double.NEGATIVE_INFINITY;
@@ -70,7 +76,10 @@ public class DoubleSummaryAggregator extends NumericSummaryAggregator<Double> {
}
}
- public static class SumDoubleAggregator implements Aggregator<Double,Double> {
+ /**
+ * Aggregator for sum operation.
+ */
+ public static class SumDoubleAggregator implements Aggregator<Double, Double> {
private CompensatedSum sum = ZERO;
@@ -81,7 +90,7 @@ public class DoubleSummaryAggregator extends NumericSummaryAggregator<Double> {
@Override
public void combine(Aggregator<Double, Double> other) {
- sum = sum.add(((SumDoubleAggregator)other).sum);
+ sum = sum.add(((SumDoubleAggregator) other).sum);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregator.java
index 5e06670..39b3e90 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregator.java
@@ -23,7 +23,7 @@ import org.apache.flink.annotation.Internal;
import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZERO;
/**
- * Aggregator that can handle Float types
+ * Aggregator that can handle Float types.
*/
@Internal
public class FloatSummaryAggregator extends NumericSummaryAggregator<Float> {
@@ -32,7 +32,10 @@ public class FloatSummaryAggregator extends NumericSummaryAggregator<Float> {
// Nested classes are only "public static" for Kryo serialization, otherwise they'd be private
- public static class MinFloatAggregator implements Aggregator<Float,Float> {
+ /**
+ * Aggregator for min operation.
+ */
+ public static class MinFloatAggregator implements Aggregator<Float, Float> {
private float min = Float.POSITIVE_INFINITY;
@@ -43,7 +46,7 @@ public class FloatSummaryAggregator extends NumericSummaryAggregator<Float> {
@Override
public void combine(Aggregator<Float, Float> other) {
- min = Math.min(min,((MinFloatAggregator)other).min);
+ min = Math.min(min, ((MinFloatAggregator) other).min);
}
@Override
@@ -52,7 +55,10 @@ public class FloatSummaryAggregator extends NumericSummaryAggregator<Float> {
}
}
- public static class MaxFloatAggregator implements Aggregator<Float,Float> {
+ /**
+ * Aggregator for max operation.
+ */
+ public static class MaxFloatAggregator implements Aggregator<Float, Float> {
private float max = Float.NEGATIVE_INFINITY;
@@ -72,7 +78,10 @@ public class FloatSummaryAggregator extends NumericSummaryAggregator<Float> {
}
}
- public static class SumFloatAggregator implements Aggregator<Float,Float> {
+ /**
+ * Aggregator for sum operation.
+ */
+ public static class SumFloatAggregator implements Aggregator<Float, Float> {
private CompensatedSum sum = ZERO;
@@ -83,7 +92,7 @@ public class FloatSummaryAggregator extends NumericSummaryAggregator<Float> {
@Override
public void combine(Aggregator<Float, Float> other) {
- sum = sum.add(((SumFloatAggregator)other).sum);
+ sum = sum.add(((SumFloatAggregator) other).sum);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregator.java
index a443019..4035444 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregator.java
@@ -21,7 +21,7 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
/**
- * Aggregator that can handle Integer types
+ * Aggregator that can handle Integer types.
*/
@Internal
public class IntegerSummaryAggregator extends NumericSummaryAggregator<Integer> {
@@ -30,7 +30,10 @@ public class IntegerSummaryAggregator extends NumericSummaryAggregator<Integer>
// Nested classes are only "public static" for Kryo serialization, otherwise they'd be private
- public static class MinIntegerAggregator implements Aggregator<Integer,Integer> {
+ /**
+ * Aggregator for min operation.
+ */
+ public static class MinIntegerAggregator implements Aggregator<Integer, Integer> {
private int min = Integer.MAX_VALUE;
@@ -41,7 +44,7 @@ public class IntegerSummaryAggregator extends NumericSummaryAggregator<Integer>
@Override
public void combine(Aggregator<Integer, Integer> other) {
- min = Math.min(min,((MinIntegerAggregator)other).min);
+ min = Math.min(min, ((MinIntegerAggregator) other).min);
}
@Override
@@ -50,7 +53,10 @@ public class IntegerSummaryAggregator extends NumericSummaryAggregator<Integer>
}
}
- public static class MaxIntegerAggregator implements Aggregator<Integer,Integer> {
+ /**
+ * Aggregator for max operation.
+ */
+ public static class MaxIntegerAggregator implements Aggregator<Integer, Integer> {
private int max = Integer.MIN_VALUE;
@@ -70,7 +76,10 @@ public class IntegerSummaryAggregator extends NumericSummaryAggregator<Integer>
}
}
- public static class SumIntegerAggregator implements Aggregator<Integer,Integer> {
+ /**
+ * Aggregator for sum operation.
+ */
+ public static class SumIntegerAggregator implements Aggregator<Integer, Integer> {
private int sum = 0;
@@ -81,7 +90,7 @@ public class IntegerSummaryAggregator extends NumericSummaryAggregator<Integer>
@Override
public void combine(Aggregator<Integer, Integer> other) {
- sum += ((SumIntegerAggregator)other).sum;
+ sum += ((SumIntegerAggregator) other).sum;
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregator.java
index 5cd314e..5940dfe 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregator.java
@@ -21,7 +21,7 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
/**
- * Aggregator that can handle Long types
+ * Aggregator that can handle Long types.
*/
@Internal
public class LongSummaryAggregator extends NumericSummaryAggregator<Long> {
@@ -30,7 +30,10 @@ public class LongSummaryAggregator extends NumericSummaryAggregator<Long> {
// Nested classes are only "public static" for Kryo serialization, otherwise they'd be private
- public static class MinLongAggregator implements Aggregator<Long,Long> {
+ /**
+ * Aggregator for min operation.
+ */
+ public static class MinLongAggregator implements Aggregator<Long, Long> {
private long min = Long.MAX_VALUE;
@@ -41,7 +44,7 @@ public class LongSummaryAggregator extends NumericSummaryAggregator<Long> {
@Override
public void combine(Aggregator<Long, Long> other) {
- min = Math.min(min,((MinLongAggregator)other).min);
+ min = Math.min(min, ((MinLongAggregator) other).min);
}
@Override
@@ -50,7 +53,10 @@ public class LongSummaryAggregator extends NumericSummaryAggregator<Long> {
}
}
- public static class MaxLongAggregator implements Aggregator<Long,Long> {
+ /**
+ * Aggregator for max operation.
+ */
+ public static class MaxLongAggregator implements Aggregator<Long, Long> {
private long max = Long.MIN_VALUE;
@@ -70,7 +76,10 @@ public class LongSummaryAggregator extends NumericSummaryAggregator<Long> {
}
}
- public static class SumLongAggregator implements Aggregator<Long,Long> {
+ /**
+ * Aggregator for sum operation.
+ */
+ private static class SumLongAggregator implements Aggregator<Long, Long> {
private long sum = 0;
@@ -81,7 +90,7 @@ public class LongSummaryAggregator extends NumericSummaryAggregator<Long> {
@Override
public void combine(Aggregator<Long, Long> other) {
- sum += ((SumLongAggregator)other).sum;
+ sum += ((SumLongAggregator) other).sum;
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/NumericSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/NumericSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/NumericSummaryAggregator.java
index a0b1ed0..4445528 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/NumericSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/NumericSummaryAggregator.java
@@ -26,18 +26,18 @@ import static org.apache.flink.api.java.summarize.aggregation.CompensatedSum.ZER
/**
* Generic aggregator for all numeric types creates a summary of a column of numbers.
*
- * Uses the Kahan summation algorithm to avoid numeric instability when computing variance.
+ * <p>Uses the Kahan summation algorithm to avoid numeric instability when computing variance.
* The algorithm is described in: "Scalable and Numerically Stable Descriptive Statistics in SystemML",
* Tian et al, International Conference on Data Engineering 2012
*
- * Implementation that couldn't be generic for all numbers was pushed to subclasses.
+ * <p>Implementation that couldn't be generic for all numbers was pushed to subclasses.
* For example, there isn't a generic way to calculate min, max, sum, isNan, isInfinite
* for all numeric types so subclasses must implement these.
*
* @param <T> numeric type to aggregrate and create a summary, e.g. Integer, DoubleValue
*/
@Internal
-public abstract class NumericSummaryAggregator<T extends Number> implements Aggregator<T,NumericColumnSummary<T>> {
+public abstract class NumericSummaryAggregator<T extends Number> implements Aggregator<T, NumericColumnSummary<T>> {
private static final long serialVersionUID = 1L;
@@ -47,21 +47,21 @@ public abstract class NumericSummaryAggregator<T extends Number> implements Aggr
private long infinityCount = 0L;
// these fields are initialized by type specific subclasses
- private Aggregator<T,T> min = initMin();
- private Aggregator<T,T> max = initMax();
- private Aggregator<T,T> sum = initSum();
+ private Aggregator<T, T> min = initMin();
+ private Aggregator<T, T> max = initMax();
+ private Aggregator<T, T> sum = initSum();
private CompensatedSum mean = ZERO;
/**
* Sum of squares of differences from the current mean (used to calculate variance).
*
- * The algorithm is described in: "Scalable and Numerically Stable Descriptive Statistics in SystemML",
+ * <p>The algorithm is described in: "Scalable and Numerically Stable Descriptive Statistics in SystemML",
* Tian et al, International Conference on Data Engineering 2012
*/
private CompensatedSum m2 = ZERO;
/**
- * Add a value to the current aggregation
+ * Add a value to the current aggregation.
*/
@Override
public void aggregate(T value) {
@@ -72,7 +72,7 @@ public abstract class NumericSummaryAggregator<T extends Number> implements Aggr
else if (isNan(value)) {
nanCount++;
}
- else if(isInfinite(value)) {
+ else if (isInfinite(value)) {
infinityCount++;
}
else {
@@ -90,7 +90,7 @@ public abstract class NumericSummaryAggregator<T extends Number> implements Aggr
}
/**
- * combine two aggregations
+ * combine two aggregations.
*/
@Override
public void combine(Aggregator<T, NumericColumnSummary<T>> otherSameType) {
@@ -130,7 +130,7 @@ public abstract class NumericSummaryAggregator<T extends Number> implements Aggr
public NumericColumnSummary<T> result() {
Double variance = null;
- if(nonMissingCount > 1) {
+ if (nonMissingCount > 1) {
variance = m2.value() / (nonMissingCount - 1);
}
@@ -152,11 +152,11 @@ public abstract class NumericSummaryAggregator<T extends Number> implements Aggr
// there isn't a generic way to calculate min, max, sum, isNan, isInfinite for all numeric types
// so subclasses must implement these
- protected abstract Aggregator<T,T> initMin();
+ protected abstract Aggregator<T, T> initMin();
- protected abstract Aggregator<T,T> initMax();
+ protected abstract Aggregator<T, T> initMax();
- protected abstract Aggregator<T,T> initSum();
+ protected abstract Aggregator<T, T> initSum();
protected abstract boolean isNan(T number);
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ObjectSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ObjectSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ObjectSummaryAggregator.java
index 532d91f..bc6aa9d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ObjectSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ObjectSummaryAggregator.java
@@ -21,8 +21,11 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.java.summarize.ObjectColumnSummary;
+/**
+ * {@link Aggregator} that calculates statistics for {@link Object} values.
+ */
@Internal
-public class ObjectSummaryAggregator implements Aggregator<Object,ObjectColumnSummary> {
+public class ObjectSummaryAggregator implements Aggregator<Object, ObjectColumnSummary> {
private long nonNullCount;
private long nullCount;
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregator.java
index a2a395d..472e472 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ShortSummaryAggregator.java
@@ -21,7 +21,7 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
/**
- * Aggregator that can handle Short types
+ * Aggregator that can handle Short types.
*/
@Internal
public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
@@ -29,14 +29,14 @@ public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
private static final long serialVersionUID = 1L;
/**
- * Like Math.min() except for shorts
+ * Like Math.min() except for shorts.
*/
public static Short min(Short a, Short b) {
return a <= b ? a : b;
}
/**
- * Like Math.max() except for shorts
+ * Like Math.max() except for shorts.
*/
public static Short max(Short a, Short b) {
return a >= b ? a : b;
@@ -44,7 +44,10 @@ public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
// Nested classes are only "public static" for Kryo serialization, otherwise they'd be private
- public static class MinShortAggregator implements Aggregator<Short,Short> {
+ /**
+ * Aggregator for min operation.
+ */
+ public static class MinShortAggregator implements Aggregator<Short, Short> {
private short min = Short.MAX_VALUE;
@@ -55,7 +58,7 @@ public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
@Override
public void combine(Aggregator<Short, Short> other) {
- min = min(min,((MinShortAggregator)other).min);
+ min = min(min, ((MinShortAggregator) other).min);
}
@Override
@@ -64,7 +67,10 @@ public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
}
}
- public static class MaxShortAggregator implements Aggregator<Short,Short> {
+ /**
+ * Aggregator for max operation.
+ */
+ public static class MaxShortAggregator implements Aggregator<Short, Short> {
private short max = Short.MIN_VALUE;
@@ -84,7 +90,10 @@ public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
}
}
- public static class SumShortAggregator implements Aggregator<Short,Short> {
+ /**
+ * Aggregator for sum operation.
+ */
+ public static class SumShortAggregator implements Aggregator<Short, Short> {
private short sum = 0;
@@ -95,7 +104,7 @@ public class ShortSummaryAggregator extends NumericSummaryAggregator<Short> {
@Override
public void combine(Aggregator<Short, Short> other) {
- sum += ((SumShortAggregator)other).sum;
+ sum += ((SumShortAggregator) other).sum;
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregator.java
index fa7d721..fdee140 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/StringSummaryAggregator.java
@@ -21,8 +21,11 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.java.summarize.StringColumnSummary;
+/**
+ * {@link Aggregator} that calculates statistics for {@link String} values.
+ */
@Internal
-public class StringSummaryAggregator implements Aggregator<String,StringColumnSummary> {
+public class StringSummaryAggregator implements Aggregator<String, StringColumnSummary> {
private long nonNullCount = 0L;
private long nullCount = 0L;
@@ -33,13 +36,13 @@ public class StringSummaryAggregator implements Aggregator<String,StringColumnSu
@Override
public void aggregate(String value) {
- if(value == null) {
+ if (value == null) {
nullCount++;
}
else {
nonNullCount++;
- if(value.isEmpty()) {
+ if (value.isEmpty()) {
emptyCount++;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactory.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactory.java
index 26e88b2..c101768 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactory.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/SummaryAggregatorFactory.java
@@ -30,7 +30,7 @@ import org.apache.flink.types.ShortValue;
import org.apache.flink.types.StringValue;
/**
- * Factory for creating Summary Aggregators
+ * Factory for creating Summary Aggregators.
*/
@Internal
public class SummaryAggregatorFactory {
@@ -46,57 +46,57 @@ public class SummaryAggregatorFactory {
}
/**
- * Create an SummaryAggregator for the supplied type
+ * Create a SummaryAggregator for the supplied type.
* @param <T> the type to aggregate
* @param <R> the result type of the aggregation
*/
@SuppressWarnings("unchecked")
- public static <T,R> Aggregator<T,R> create(Class<T> type) {
+ public static <T, R> Aggregator<T, R> create(Class<T> type) {
if (type == Long.class) {
- return (Aggregator<T,R>) new LongSummaryAggregator();
+ return (Aggregator<T, R>) new LongSummaryAggregator();
}
else if (type == LongValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.LongValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.LongValueSummaryAggregator();
}
else if (type == Integer.class) {
- return (Aggregator<T,R>) new IntegerSummaryAggregator();
+ return (Aggregator<T, R>) new IntegerSummaryAggregator();
}
else if (type == IntValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.IntegerValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.IntegerValueSummaryAggregator();
}
else if (type == Double.class) {
- return (Aggregator<T,R>) new DoubleSummaryAggregator();
+ return (Aggregator<T, R>) new DoubleSummaryAggregator();
}
else if (type == DoubleValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.DoubleValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.DoubleValueSummaryAggregator();
}
else if (type == Float.class) {
- return (Aggregator<T,R>) new FloatSummaryAggregator();
+ return (Aggregator<T, R>) new FloatSummaryAggregator();
}
else if (type == FloatValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.FloatValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.FloatValueSummaryAggregator();
}
else if (type == Short.class) {
- return (Aggregator<T,R>) new ShortSummaryAggregator();
+ return (Aggregator<T, R>) new ShortSummaryAggregator();
}
else if (type == ShortValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.ShortValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.ShortValueSummaryAggregator();
}
else if (type == Boolean.class) {
- return (Aggregator<T,R>) new BooleanSummaryAggregator();
+ return (Aggregator<T, R>) new BooleanSummaryAggregator();
}
else if (type == BooleanValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.BooleanValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.BooleanValueSummaryAggregator();
}
else if (type == String.class) {
- return (Aggregator<T,R>) new StringSummaryAggregator();
+ return (Aggregator<T, R>) new StringSummaryAggregator();
}
else if (type == StringValue.class) {
- return (Aggregator<T,R>) new ValueSummaryAggregator.StringValueSummaryAggregator();
+ return (Aggregator<T, R>) new ValueSummaryAggregator.StringValueSummaryAggregator();
}
else {
// rather than error for unsupported types do something very generic
- return (Aggregator<T,R>) new ObjectSummaryAggregator();
+ return (Aggregator<T, R>) new ObjectSummaryAggregator();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/TupleSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/TupleSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/TupleSummaryAggregator.java
index a75e582..55fa079 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/TupleSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/TupleSummaryAggregator.java
@@ -25,7 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple;
* Aggregate tuples using an array of aggregators, one for each "column" or position within the Tuple.
*/
@Internal
-public class TupleSummaryAggregator<R extends Tuple> implements Aggregator<Tuple,R> {
+public class TupleSummaryAggregator<R extends Tuple> implements Aggregator<Tuple, R> {
private static final long serialVersionUID = 1L;
@@ -38,7 +38,7 @@ public class TupleSummaryAggregator<R extends Tuple> implements Aggregator<Tuple
@Override
@SuppressWarnings("unchecked")
public void aggregate(Tuple value) {
- for(int i = 0; i < columnAggregators.length; i++) {
+ for (int i = 0; i < columnAggregators.length; i++) {
columnAggregators[i].aggregate(value.getField(i));
}
@@ -48,7 +48,7 @@ public class TupleSummaryAggregator<R extends Tuple> implements Aggregator<Tuple
@SuppressWarnings("unchecked")
public void combine(Aggregator<Tuple, R> other) {
TupleSummaryAggregator tupleSummaryAggregator = (TupleSummaryAggregator) other;
- for( int i = 0; i < columnAggregators.length; i++) {
+ for (int i = 0; i < columnAggregators.length; i++) {
columnAggregators[i].combine(tupleSummaryAggregator.columnAggregators[i]);
}
}
@@ -59,7 +59,7 @@ public class TupleSummaryAggregator<R extends Tuple> implements Aggregator<Tuple
try {
Class tupleClass = Tuple.getTupleClass(columnAggregators.length);
R tuple = (R) tupleClass.newInstance();
- for(int i = 0; i < columnAggregators.length; i++) {
+ for (int i = 0; i < columnAggregators.length; i++) {
tuple.setField(columnAggregators[i].result(), i);
}
return tuple;
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ValueSummaryAggregator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ValueSummaryAggregator.java b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ValueSummaryAggregator.java
index 4e55dd0..ab49144 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ValueSummaryAggregator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/summarize/aggregation/ValueSummaryAggregator.java
@@ -37,11 +37,11 @@ import org.apache.flink.types.Value;
*
* @param <VT> the "Value Type" to aggregate, e.g. DoubleValue, StringValue
* @param <PT> the "Primitive Type" that "Value Type" can be naturally converted to, e.g. DoubleValue converts to Double
- * @param <R> the result type of the aggregation, e.g. NumericColumnSummary<Double>
+ * @param <R> the result type of the aggregation, e.g. NumericColumnSummary<Double>
* @param <A> the underlying primitive Aggregator that does the actual work, e.g. DoubleSummaryAggregator
*/
@Internal
-public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Aggregator<PT,R>> implements Aggregator<VT,R> {
+public abstract class ValueSummaryAggregator<VT extends Value, PT, R, A extends Aggregator<PT, R>> implements Aggregator<VT, R> {
private A aggregator = initPrimitiveAggregator();
@@ -57,8 +57,8 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
@Override
@SuppressWarnings("unchecked")
- public void combine(Aggregator<VT,R> otherSameType) {
- ValueSummaryAggregator<VT,PT,R,A> other = (ValueSummaryAggregator<VT,PT,R,A>) otherSameType;
+ public void combine(Aggregator<VT, R> otherSameType) {
+ ValueSummaryAggregator<VT, PT, R, A> other = (ValueSummaryAggregator<VT, PT, R, A>) otherSameType;
aggregator.combine(other.aggregator);
}
@@ -70,7 +70,7 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
/**
* Initialize an aggregator that can be used for the underlying primitive in the Value type.
*
- * E.g. DoubleValues can easily be converted to Double and could use an underlying Aggregator<Double,?>
+ * <p>E.g. DoubleValues can easily be converted to Double and could use an underlying Aggregator<Double,?>
*/
protected abstract A initPrimitiveAggregator();
@@ -79,12 +79,14 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
*/
protected abstract PT getValue(VT value);
-
// -----------------------------------------------------------------------------
// Implementations below
// -----------------------------------------------------------------------------
- public static class ShortValueSummaryAggregator extends ValueSummaryAggregator<ShortValue,Short,NumericColumnSummary<Short>,ShortSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link Short}.
+ */
+ public static class ShortValueSummaryAggregator extends ValueSummaryAggregator<ShortValue, Short, NumericColumnSummary<Short>, ShortSummaryAggregator> {
@Override
protected ShortSummaryAggregator initPrimitiveAggregator() {
@@ -97,7 +99,10 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
}
}
- public static class IntegerValueSummaryAggregator extends ValueSummaryAggregator<IntValue,Integer,NumericColumnSummary<Integer>,IntegerSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link Integer}.
+ */
+ public static class IntegerValueSummaryAggregator extends ValueSummaryAggregator<IntValue, Integer, NumericColumnSummary<Integer>, IntegerSummaryAggregator> {
@Override
protected IntegerSummaryAggregator initPrimitiveAggregator() {
@@ -110,7 +115,10 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
}
}
- public static class LongValueSummaryAggregator extends ValueSummaryAggregator<LongValue,Long,NumericColumnSummary<Long>,LongSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link Long}.
+ */
+ public static class LongValueSummaryAggregator extends ValueSummaryAggregator<LongValue, Long, NumericColumnSummary<Long>, LongSummaryAggregator> {
@Override
protected LongSummaryAggregator initPrimitiveAggregator() {
@@ -123,7 +131,10 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
}
}
- public static class FloatValueSummaryAggregator extends ValueSummaryAggregator<FloatValue,Float,NumericColumnSummary<Float>,FloatSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link Float}.
+ */
+ public static class FloatValueSummaryAggregator extends ValueSummaryAggregator<FloatValue, Float, NumericColumnSummary<Float>, FloatSummaryAggregator> {
@Override
protected FloatSummaryAggregator initPrimitiveAggregator() {
@@ -136,7 +147,10 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
}
}
- public static class DoubleValueSummaryAggregator extends ValueSummaryAggregator<DoubleValue,Double,NumericColumnSummary<Double>,DoubleSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link Double}.
+ */
+ public static class DoubleValueSummaryAggregator extends ValueSummaryAggregator<DoubleValue, Double, NumericColumnSummary<Double>, DoubleSummaryAggregator> {
@Override
protected DoubleSummaryAggregator initPrimitiveAggregator() {
@@ -149,7 +163,10 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
}
}
- public static class BooleanValueSummaryAggregator extends ValueSummaryAggregator<BooleanValue,Boolean,BooleanColumnSummary,BooleanSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link Boolean}.
+ */
+ public static class BooleanValueSummaryAggregator extends ValueSummaryAggregator<BooleanValue, Boolean, BooleanColumnSummary, BooleanSummaryAggregator> {
@Override
protected BooleanSummaryAggregator initPrimitiveAggregator() {
@@ -162,7 +179,10 @@ public abstract class ValueSummaryAggregator<VT extends Value,PT,R, A extends Ag
}
}
- public static class StringValueSummaryAggregator extends ValueSummaryAggregator<StringValue,String,StringColumnSummary,StringSummaryAggregator> {
+ /**
+ * A {@link ValueSummaryAggregator} for {@link String}.
+ */
+ public static class StringValueSummaryAggregator extends ValueSummaryAggregator<StringValue, String, StringColumnSummary, StringSummaryAggregator> {
@Override
protected StringSummaryAggregator initPrimitiveAggregator() {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java
index abb36c3..c576125 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/AggregateCombineHarness.java
@@ -27,7 +27,7 @@ import java.util.List;
* This harness uses multiple aggregators and variously aggregates and combines against
* a list of values while calling a compareResults() method.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways but can help uncover various kinds of bugs that can show
* up in aggregators.
*
@@ -35,12 +35,12 @@ import java.util.List;
* @param <R> the type of the results of the aggregation
* @param <A> the aggregator to use
*/
-public abstract class AggregateCombineHarness<T,R,A extends Aggregator<T,R>> {
+public abstract class AggregateCombineHarness<T, R, A extends Aggregator<T, R>> {
/**
* Compare results from different runs of aggregate/combine to make sure they are the same.
*
- * Subclasses should cause an Assertion failure or throw an Exception if the results aren't
+ * <p>Subclasses should cause an Assertion failure or throw an Exception if the results aren't
* equal or at least close enough.
*/
protected abstract void compareResults(R result1, R result2);
@@ -50,7 +50,7 @@ public abstract class AggregateCombineHarness<T,R,A extends Aggregator<T,R>> {
*/
@SafeVarargs
public final R summarize(T... values) {
- if (values.length == 0 ) {
+ if (values.length == 0) {
// when there is nothing to aggregate just combine two empty aggregators and get the result.
A agg1 = initAggregator();
agg1.combine(initAggregator());
@@ -65,7 +65,7 @@ public abstract class AggregateCombineHarness<T,R,A extends Aggregator<T,R>> {
List<T> list = Arrays.asList(values);
Collections.shuffle(list);
- for (int i = 0; i < values.length; i++ ) {
+ for (int i = 0; i < values.length; i++) {
// Two aggregators are used so that combine() can be tested also.
// It shouldn't matter which aggregator is used because they are combined at the end so
@@ -100,7 +100,7 @@ public abstract class AggregateCombineHarness<T,R,A extends Aggregator<T,R>> {
try {
// Instantiate a generic type
// http://stackoverflow.com/questions/75175/create-instance-of-generic-type-in-java
- return (A) ((Class)((ParameterizedType)this.getClass().getGenericSuperclass()).getActualTypeArguments()[2]).newInstance();
+ return (A) ((Class) ((ParameterizedType) this.getClass().getGenericSuperclass()).getActualTypeArguments()[2]).newInstance();
}
catch (Exception e) {
throw new RuntimeException("Could not initialize aggregator", e);
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java
index 4340643..e046dc9 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanSummaryAggregatorTest.java
@@ -19,10 +19,13 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.BooleanColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
-
+/**
+ * Tests for {@link BooleanSummaryAggregator}.
+ */
public class BooleanSummaryAggregatorTest {
@Test
@@ -68,7 +71,7 @@ public class BooleanSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways.
*/
protected BooleanColumnSummary summarize(Boolean... values) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java
index b589681..8c7f94b 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/BooleanValueSummaryAggregatorTest.java
@@ -20,20 +20,24 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.BooleanColumnSummary;
import org.apache.flink.types.BooleanValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.BooleanValueSummaryAggregator}.
+ */
public class BooleanValueSummaryAggregatorTest extends BooleanSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways.
*/
protected BooleanColumnSummary summarize(Boolean... values) {
BooleanValue[] booleanValues = new BooleanValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
booleanValues[i] = new BooleanValue(values[i]);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java
index 5036123..6d086d9 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/CompensatedSumTest.java
@@ -21,12 +21,15 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.junit.Assert;
import org.junit.Test;
+/**
+ * Tests for {@link CompensatedSum}.
+ */
public class CompensatedSumTest {
/**
* When adding a series of numbers the order of the numbers should not impact the results.
*
- * This test shows that a naive summation comes up with a different result than Kahan
+ * <p>This test shows that a naive summation comes up with a different result than Kahan
* Summation when you start with either a smaller or larger number in some cases and
* helps prove our Kahan Summation is working.
*/
@@ -40,7 +43,7 @@ public class CompensatedSumTest {
double naiveResult1 = smallSum.value();
double naiveResult2 = largeSum.value();
- for(int i = 0; i < 10; i++) {
+ for (int i = 0; i < 10; i++) {
compensatedResult1 = compensatedResult1.add(smallSum);
compensatedResult2 = compensatedResult2.add(smallSum);
naiveResult1 += smallSum.value();
@@ -68,7 +71,7 @@ public class CompensatedSumTest {
@Test
public void testDelta() throws Exception {
CompensatedSum compensatedResult1 = new CompensatedSum(0.001, 0.0);
- for(int i = 0; i < 10; i++) {
+ for (int i = 0; i < 10; i++) {
compensatedResult1 = compensatedResult1.add(0.001);
}
Assert.assertEquals(0.011, compensatedResult1.value(), 0.0);
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java
index 0ef3969..3babf2c 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleSummaryAggregatorTest.java
@@ -19,17 +19,21 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
+/**
+ * Tests for {@link DoubleSummaryAggregator}.
+ */
public class DoubleSummaryAggregatorTest {
/**
* Use some values from Anscombe's Quartet for testing.
*
- * There was no particular reason to use these except they have known means and variance.
+ * <p>There was no particular reason to use these except they have known means and variance.
*
- * https://en.wikipedia.org/wiki/Anscombe%27s_quartet
+ * <p>https://en.wikipedia.org/wiki/Anscombe%27s_quartet
*/
@Test
public void testAnscomesQuartetXValues() throws Exception {
@@ -54,9 +58,9 @@ public class DoubleSummaryAggregatorTest {
/**
* Use some values from Anscombe's Quartet for testing.
*
- * There was no particular reason to use these except they have known means and variance.
+ * <p>There was no particular reason to use these except they have known means and variance.
*
- * https://en.wikipedia.org/wiki/Anscombe%27s_quartet
+ * <p>https://en.wikipedia.org/wiki/Anscombe%27s_quartet
*/
@Test
public void testAnscomesQuartetYValues() throws Exception {
@@ -156,11 +160,11 @@ public class DoubleSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways.
*/
protected NumericColumnSummary<Double> summarize(Double... values) {
- return new AggregateCombineHarness<Double,NumericColumnSummary<Double>,DoubleSummaryAggregator>() {
+ return new AggregateCombineHarness<Double, NumericColumnSummary<Double>, DoubleSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Double> result1, NumericColumnSummary<Double> result2) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java
index a30d6aa..36bac75 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/DoubleValueSummaryAggregatorTest.java
@@ -20,26 +20,30 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
import org.apache.flink.types.DoubleValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.DoubleValueSummaryAggregator}.
+ */
public class DoubleValueSummaryAggregatorTest extends DoubleSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating and combining
+ * <p>This method breaks the rule of "testing only one thing" by aggregating and combining
* a bunch of different ways.
*/
protected NumericColumnSummary<Double> summarize(Double... values) {
DoubleValue[] doubleValues = new DoubleValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
doubleValues[i] = new DoubleValue(values[i]);
}
}
- return new AggregateCombineHarness<DoubleValue,NumericColumnSummary<Double>,ValueSummaryAggregator.DoubleValueSummaryAggregator>() {
+ return new AggregateCombineHarness<DoubleValue, NumericColumnSummary<Double>, ValueSummaryAggregator.DoubleValueSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Double> result1, NumericColumnSummary<Double> result2) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java
index 3e0c899..b1fc2a7 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatSummaryAggregatorTest.java
@@ -19,18 +19,21 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
-
+/**
+ * Tests for {@link FloatSummaryAggregator}.
+ */
public class FloatSummaryAggregatorTest {
/**
* Use some values from Anscombe's Quartet for testing.
*
- * There was no particular reason to use these except they have known means and variance.
+ * <p>There was no particular reason to use these except they have known means and variance.
*
- * https://en.wikipedia.org/wiki/Anscombe%27s_quartet
+ * <p>https://en.wikipedia.org/wiki/Anscombe%27s_quartet
*/
@Test
public void testAnscomesQuartetXValues() throws Exception {
@@ -55,9 +58,9 @@ public class FloatSummaryAggregatorTest {
/**
* Use some values from Anscombe's Quartet for testing.
*
- * There was no particular reason to use these except they have known means and variance.
+ * <p>There was no particular reason to use these except they have known means and variance.
*
- * https://en.wikipedia.org/wiki/Anscombe%27s_quartet
+ * <p>https://en.wikipedia.org/wiki/Anscombe%27s_quartet
*/
@Test
public void testAnscomesQuartetYValues() throws Exception {
@@ -145,12 +148,12 @@ public class FloatSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating
+ * <p>This method breaks the rule of "testing only one thing" by aggregating
* and combining a bunch of different ways.
*/
protected NumericColumnSummary<Float> summarize(Float... values) {
- return new AggregateCombineHarness<Float,NumericColumnSummary<Float>,FloatSummaryAggregator>() {
+ return new AggregateCombineHarness<Float, NumericColumnSummary<Float>, FloatSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Float> result1, NumericColumnSummary<Float> result2) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java
index ff87946..a511838 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/FloatValueSummaryAggregatorTest.java
@@ -20,27 +20,31 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
import org.apache.flink.types.FloatValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.FloatValueSummaryAggregator}.
+ */
public class FloatValueSummaryAggregatorTest extends FloatSummaryAggregatorTest {
/**
* Helper method for summarizing a list of values.
*
- * This method breaks the rule of "testing only one thing" by aggregating
+ * <p>This method breaks the rule of "testing only one thing" by aggregating
* and combining a bunch of different ways.
*/
@Override
protected NumericColumnSummary<Float> summarize(Float... values) {
FloatValue[] floatValues = new FloatValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
floatValues[i] = new FloatValue(values[i]);
}
}
- return new AggregateCombineHarness<FloatValue,NumericColumnSummary<Float>,ValueSummaryAggregator.FloatValueSummaryAggregator>() {
+ return new AggregateCombineHarness<FloatValue, NumericColumnSummary<Float>, ValueSummaryAggregator.FloatValueSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Float> result1, NumericColumnSummary<Float> result2) {
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java
index 110d2cc..4e36ad0 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerSummaryAggregatorTest.java
@@ -19,9 +19,13 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
+/**
+ * Tests for {@link IntegerSummaryAggregator}.
+ */
public class IntegerSummaryAggregatorTest {
@Test
@@ -85,31 +89,31 @@ public class IntegerSummaryAggregatorTest {
}
/**
- * Helper method for summarizing a list of values
+ * Helper method for summarizing a list of values.
*/
protected NumericColumnSummary<Integer> summarize(Integer... values) {
- return new AggregateCombineHarness<Integer,NumericColumnSummary<Integer>,IntegerSummaryAggregator>() {
+ return new AggregateCombineHarness<Integer, NumericColumnSummary<Integer>, IntegerSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Integer> result1, NumericColumnSummary<Integer> result2) {
- Assert.assertEquals(result1.getTotalCount(),result2.getTotalCount());
- Assert.assertEquals(result1.getNullCount(),result2.getNullCount());
- Assert.assertEquals(result1.getMissingCount(),result2.getMissingCount());
- Assert.assertEquals(result1.getNonMissingCount(),result2.getNonMissingCount());
- Assert.assertEquals(result1.getInfinityCount(),result2.getInfinityCount());
- Assert.assertEquals(result1.getNanCount(),result2.getNanCount());
+ Assert.assertEquals(result1.getTotalCount(), result2.getTotalCount());
+ Assert.assertEquals(result1.getNullCount(), result2.getNullCount());
+ Assert.assertEquals(result1.getMissingCount(), result2.getMissingCount());
+ Assert.assertEquals(result1.getNonMissingCount(), result2.getNonMissingCount());
+ Assert.assertEquals(result1.getInfinityCount(), result2.getInfinityCount());
+ Assert.assertEquals(result1.getNanCount(), result2.getNanCount());
- Assert.assertEquals(result1.containsNull(),result2.containsNull());
- Assert.assertEquals(result1.containsNonNull(),result2.containsNonNull());
+ Assert.assertEquals(result1.containsNull(), result2.containsNull());
+ Assert.assertEquals(result1.containsNonNull(), result2.containsNonNull());
- Assert.assertEquals(result1.getMin().intValue(),result2.getMin().intValue());
+ Assert.assertEquals(result1.getMin().intValue(), result2.getMin().intValue());
Assert.assertEquals(result1.getMax().intValue(), result2.getMax().intValue());
- Assert.assertEquals(result1.getSum().intValue(),result2.getSum().intValue());
+ Assert.assertEquals(result1.getSum().intValue(), result2.getSum().intValue());
Assert.assertEquals(result1.getMean().doubleValue(), result2.getMean().doubleValue(), 1e-12d);
- Assert.assertEquals(result1.getVariance().doubleValue(),result2.getVariance().doubleValue(), 1e-9d);
- Assert.assertEquals(result1.getStandardDeviation().doubleValue(),result2.getStandardDeviation().doubleValue(), 1e-12d);
+ Assert.assertEquals(result1.getVariance().doubleValue(), result2.getVariance().doubleValue(), 1e-9d);
+ Assert.assertEquals(result1.getStandardDeviation().doubleValue(), result2.getStandardDeviation().doubleValue(), 1e-12d);
}
}.summarize(values);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java
index 6ac5485..40c8bdb 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/IntegerValueSummaryAggregatorTest.java
@@ -20,41 +20,45 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
import org.apache.flink.types.IntValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.IntegerValueSummaryAggregator}.
+ */
public class IntegerValueSummaryAggregatorTest extends IntegerSummaryAggregatorTest {
@Override
protected NumericColumnSummary<Integer> summarize(Integer... values) {
IntValue[] intValues = new IntValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
intValues[i] = new IntValue(values[i]);
}
}
- return new AggregateCombineHarness<IntValue,NumericColumnSummary<Integer>,ValueSummaryAggregator.IntegerValueSummaryAggregator>() {
+ return new AggregateCombineHarness<IntValue, NumericColumnSummary<Integer>, ValueSummaryAggregator.IntegerValueSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Integer> result1, NumericColumnSummary<Integer> result2) {
Assert.assertEquals(result1.getTotalCount(), result2.getTotalCount());
- Assert.assertEquals(result1.getNullCount(),result2.getNullCount());
- Assert.assertEquals(result1.getMissingCount(),result2.getMissingCount());
- Assert.assertEquals(result1.getNonMissingCount(),result2.getNonMissingCount());
- Assert.assertEquals(result1.getInfinityCount(),result2.getInfinityCount());
- Assert.assertEquals(result1.getNanCount(),result2.getNanCount());
+ Assert.assertEquals(result1.getNullCount(), result2.getNullCount());
+ Assert.assertEquals(result1.getMissingCount(), result2.getMissingCount());
+ Assert.assertEquals(result1.getNonMissingCount(), result2.getNonMissingCount());
+ Assert.assertEquals(result1.getInfinityCount(), result2.getInfinityCount());
+ Assert.assertEquals(result1.getNanCount(), result2.getNanCount());
- Assert.assertEquals(result1.containsNull(),result2.containsNull());
- Assert.assertEquals(result1.containsNonNull(),result2.containsNonNull());
+ Assert.assertEquals(result1.containsNull(), result2.containsNull());
+ Assert.assertEquals(result1.containsNonNull(), result2.containsNonNull());
- Assert.assertEquals(result1.getMin().intValue(),result2.getMin().intValue());
+ Assert.assertEquals(result1.getMin().intValue(), result2.getMin().intValue());
Assert.assertEquals(result1.getMax().intValue(), result2.getMax().intValue());
- Assert.assertEquals(result1.getSum().intValue(),result2.getSum().intValue());
+ Assert.assertEquals(result1.getSum().intValue(), result2.getSum().intValue());
Assert.assertEquals(result1.getMean().doubleValue(), result2.getMean().doubleValue(), 1e-12d);
- Assert.assertEquals(result1.getVariance().doubleValue(),result2.getVariance().doubleValue(), 1e-9d);
- Assert.assertEquals(result1.getStandardDeviation().doubleValue(),result2.getStandardDeviation().doubleValue(), 1e-12d);
+ Assert.assertEquals(result1.getVariance().doubleValue(), result2.getVariance().doubleValue(), 1e-9d);
+ Assert.assertEquals(result1.getStandardDeviation().doubleValue(), result2.getStandardDeviation().doubleValue(), 1e-12d);
}
}.summarize(intValues);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java
index 1905657..8e0c909 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongSummaryAggregatorTest.java
@@ -19,9 +19,13 @@
package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
+
import org.junit.Assert;
import org.junit.Test;
+/**
+ * Tests for {@link LongSummaryAggregator}.
+ */
public class LongSummaryAggregatorTest {
@Test
@@ -83,30 +87,30 @@ public class LongSummaryAggregatorTest {
}
/**
- * Helper method for summarizing a list of values
+ * Helper method for summarizing a list of values.
*/
protected NumericColumnSummary<Long> summarize(Long... values) {
- return new AggregateCombineHarness<Long,NumericColumnSummary<Long>,LongSummaryAggregator>() {
+ return new AggregateCombineHarness<Long, NumericColumnSummary<Long>, LongSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Long> result1, NumericColumnSummary<Long> result2) {
- Assert.assertEquals(result1.getTotalCount(),result2.getTotalCount());
+ Assert.assertEquals(result1.getTotalCount(), result2.getTotalCount());
Assert.assertEquals(result1.getNullCount(), result2.getNullCount());
- Assert.assertEquals(result1.getMissingCount(),result2.getMissingCount());
- Assert.assertEquals(result1.getNonMissingCount(),result2.getNonMissingCount());
- Assert.assertEquals(result1.getInfinityCount(),result2.getInfinityCount());
- Assert.assertEquals(result1.getNanCount(),result2.getNanCount());
+ Assert.assertEquals(result1.getMissingCount(), result2.getMissingCount());
+ Assert.assertEquals(result1.getNonMissingCount(), result2.getNonMissingCount());
+ Assert.assertEquals(result1.getInfinityCount(), result2.getInfinityCount());
+ Assert.assertEquals(result1.getNanCount(), result2.getNanCount());
Assert.assertEquals(result1.containsNull(), result2.containsNull());
- Assert.assertEquals(result1.containsNonNull(),result2.containsNonNull());
+ Assert.assertEquals(result1.containsNonNull(), result2.containsNonNull());
- Assert.assertEquals(result1.getMin().longValue(),result2.getMin().longValue());
+ Assert.assertEquals(result1.getMin().longValue(), result2.getMin().longValue());
Assert.assertEquals(result1.getMax().longValue(), result2.getMax().longValue());
- Assert.assertEquals(result1.getSum().longValue(),result2.getSum().longValue());
+ Assert.assertEquals(result1.getSum().longValue(), result2.getSum().longValue());
Assert.assertEquals(result1.getMean().doubleValue(), result2.getMean().doubleValue(), 1e-12d);
- Assert.assertEquals(result1.getVariance().doubleValue(),result2.getVariance().doubleValue(), 1e-9d);
- Assert.assertEquals(result1.getStandardDeviation().doubleValue(),result2.getStandardDeviation().doubleValue(), 1e-12d);
+ Assert.assertEquals(result1.getVariance().doubleValue(), result2.getVariance().doubleValue(), 1e-9d);
+ Assert.assertEquals(result1.getStandardDeviation().doubleValue(), result2.getStandardDeviation().doubleValue(), 1e-12d);
}
}.summarize(values);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8bd537fd/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java
index eecda69..7164bfa 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/summarize/aggregation/LongValueSummaryAggregatorTest.java
@@ -20,44 +20,48 @@ package org.apache.flink.api.java.summarize.aggregation;
import org.apache.flink.api.java.summarize.NumericColumnSummary;
import org.apache.flink.types.LongValue;
+
import org.junit.Assert;
+/**
+ * Tests for {@link ValueSummaryAggregator.LongValueSummaryAggregator}.
+ */
public class LongValueSummaryAggregatorTest extends LongSummaryAggregatorTest {
/**
- * Helper method for summarizing a list of values
+ * Helper method for summarizing a list of values.
*/
@Override
protected NumericColumnSummary<Long> summarize(Long... values) {
LongValue[] longValues = new LongValue[values.length];
- for(int i = 0; i < values.length; i++) {
+ for (int i = 0; i < values.length; i++) {
if (values[i] != null) {
longValues[i] = new LongValue(values[i]);
}
}
- return new AggregateCombineHarness<LongValue,NumericColumnSummary<Long>,ValueSummaryAggregator.LongValueSummaryAggregator>() {
+ return new AggregateCombineHarness<LongValue, NumericColumnSummary<Long>, ValueSummaryAggregator.LongValueSummaryAggregator>() {
@Override
protected void compareResults(NumericColumnSummary<Long> result1, NumericColumnSummary<Long> result2) {
- Assert.assertEquals(result1.getTotalCount(),result2.getTotalCount());
+ Assert.assertEquals(result1.getTotalCount(), result2.getTotalCount());
Assert.assertEquals(result1.getNullCount(), result2.getNullCount());
- Assert.assertEquals(result1.getMissingCount(),result2.getMissingCount());
- Assert.assertEquals(result1.getNonMissingCount(),result2.getNonMissingCount());
- Assert.assertEquals(result1.getInfinityCount(),result2.getInfinityCount());
- Assert.assertEquals(result1.getNanCount(),result2.getNanCount());
+ Assert.assertEquals(result1.getMissingCount(), result2.getMissingCount());
+ Assert.assertEquals(result1.getNonMissingCount(), result2.getNonMissingCount());
+ Assert.assertEquals(result1.getInfinityCount(), result2.getInfinityCount());
+ Assert.assertEquals(result1.getNanCount(), result2.getNanCount());
Assert.assertEquals(result1.containsNull(), result2.containsNull());
- Assert.assertEquals(result1.containsNonNull(),result2.containsNonNull());
+ Assert.assertEquals(result1.containsNonNull(), result2.containsNonNull());
- Assert.assertEquals(result1.getMin().longValue(),result2.getMin().longValue());
+ Assert.assertEquals(result1.getMin().longValue(), result2.getMin().longValue());
Assert.assertEquals(result1.getMax().longValue(), result2.getMax().longValue());
- Assert.assertEquals(result1.getSum().longValue(),result2.getSum().longValue());
+ Assert.assertEquals(result1.getSum().longValue(), result2.getSum().longValue());
Assert.assertEquals(result1.getMean().doubleValue(), result2.getMean().doubleValue(), 1e-12d);
- Assert.assertEquals(result1.getVariance().doubleValue(),result2.getVariance().doubleValue(), 1e-9d);
- Assert.assertEquals(result1.getStandardDeviation().doubleValue(),result2.getStandardDeviation().doubleValue(), 1e-12d);
+ Assert.assertEquals(result1.getVariance().doubleValue(), result2.getVariance().doubleValue(), 1e-9d);
+ Assert.assertEquals(result1.getStandardDeviation().doubleValue(), result2.getStandardDeviation().doubleValue(), 1e-12d);
}
}.summarize(longValues);
}
[08/12] flink git commit: [FLINK-7253] [tests] Remove
CommonTestUtils#assumeJava8
Posted by ch...@apache.org.
[FLINK-7253] [tests] Remove CommonTestUtils#assumeJava8
This closes #4400.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/a4a4e0b9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/a4a4e0b9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/a4a4e0b9
Branch: refs/heads/master
Commit: a4a4e0b9e5eff3d578fde7e2fca3e84661d84b40
Parents: 82c8e18
Author: zentol <ch...@apache.org>
Authored: Mon Jul 24 13:24:30 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:09 2017 +0200
----------------------------------------------------------------------
.../cassandra/CassandraConnectorITCase.java | 3 --
.../runtime/util/BlockingShutdownTest.java | 3 --
.../runtime/util/JvmExitOnFatalErrorTest.java | 3 --
.../flink/core/testutils/CommonTestUtils.java | 30 --------------------
4 files changed, 39 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/a4a4e0b9/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
index fe538a8..bc5d1a8 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
@@ -131,9 +131,6 @@ public class CassandraConnectorITCase extends WriteAheadSinkTestBase<Tuple3<Stri
@BeforeClass
public static void startCassandra() throws IOException {
- // check if we should run this test, current Cassandra version requires Java >= 1.8
- org.apache.flink.core.testutils.CommonTestUtils.assumeJava8();
-
// generate temporary files
tmpDir = CommonTestUtils.createTempDirectory();
ClassLoader classLoader = CassandraConnectorITCase.class.getClassLoader();
http://git-wip-us.apache.org/repos/asf/flink/blob/a4a4e0b9/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java
index f22f42f..1fdb4f2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java
@@ -46,9 +46,6 @@ public class BlockingShutdownTest {
// this test works only on linux
assumeTrue(OperatingSystem.isLinux());
- // this test leaves remaining processes if not executed with Java 8
- CommonTestUtils.assumeJava8();
-
final File markerFile = new File(
EnvironmentInformation.getTemporaryFileDirectory(), UUID.randomUUID() + ".marker");
http://git-wip-us.apache.org/repos/asf/flink/blob/a4a4e0b9/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
index c78a3d5..8031b22 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java
@@ -83,9 +83,6 @@ public class JvmExitOnFatalErrorTest {
// this test works only on linux
assumeTrue(OperatingSystem.isLinux());
- // this test leaves remaining processes if not executed with Java 8
- CommonTestUtils.assumeJava8();
-
// to check what went wrong (when the test hangs) uncomment this line
// ProcessEntryPoint.main(new String[0]);
http://git-wip-us.apache.org/repos/asf/flink/blob/a4a4e0b9/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
index f0abf40..ecaed1a 100644
--- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
+++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java
@@ -18,9 +18,6 @@
package org.apache.flink.core.testutils;
-import org.junit.Assume;
-import org.junit.internal.AssumptionViolatedException;
-
import java.io.BufferedWriter;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -37,8 +34,6 @@ import java.security.ProtectionDomain;
import java.security.cert.Certificate;
import java.util.Map;
-import static org.junit.Assert.fail;
-
/**
* This class contains reusable utility methods for unit tests.
*/
@@ -119,31 +114,6 @@ public class CommonTestUtils {
}
// ------------------------------------------------------------------------
- // Preconditions on the test environment
- // ------------------------------------------------------------------------
-
- /**
- * Checks whether this code runs in a Java 8 (Java 1.8) JVM. If not, this throws a
- * {@link AssumptionViolatedException}, which causes JUnit to skip the test that
- * called this method.
- */
- public static void assumeJava8() {
- try {
- String javaVersionString = System.getProperty("java.runtime.version").substring(0, 3);
- float javaVersion = Float.parseFloat(javaVersionString);
- Assume.assumeTrue(javaVersion >= 1.8f);
- }
- catch (AssumptionViolatedException e) {
- System.out.println("Skipping CassandraConnectorITCase, because the JDK is < Java 8+");
- throw e;
- }
- catch (Exception e) {
- e.printStackTrace();
- fail("Cannot determine Java version: " + e.getMessage());
- }
- }
-
- // ------------------------------------------------------------------------
// Manipulation of environment
// ------------------------------------------------------------------------
[10/12] flink git commit: [hotfix] [tests] minor test improvements in
TaskManagerConfigurationTest
Posted by ch...@apache.org.
[hotfix] [tests] minor test improvements in TaskManagerConfigurationTest
* use a proper JUnit temporary folder that ensures uniqueness
* do not catch an exception just to fail with its message - if we let it throw,
we do get more information out of the failure
* also re-set the default file system scheme (a static member!) after finishing
the test
This closes #4401.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d4b7ff35
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d4b7ff35
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d4b7ff35
Branch: refs/heads/master
Commit: d4b7ff35ddd2514945e83b84a597fa0654456516
Parents: 8bd537f
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Wed Jul 26 12:02:59 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:09 2017 +0200
----------------------------------------------------------------------
.../TaskManagerConfigurationTest.java | 39 +++++++++++---------
1 file changed, 21 insertions(+), 18 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d4b7ff35/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
index 96d1455..e73c684 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerConfigurationTest.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.taskmanager;
+import net.jcip.annotations.NotThreadSafe;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
@@ -25,12 +26,13 @@ import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.testutils.CommonTestUtils;
import org.apache.flink.runtime.concurrent.Executors;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
import scala.Tuple2;
import java.io.File;
@@ -39,16 +41,23 @@ import java.io.PrintWriter;
import java.lang.reflect.Field;
import java.net.*;
import java.util.Iterator;
-import java.util.UUID;
import static org.junit.Assert.*;
/**
* Validates that the TaskManager startup properly obeys the configuration
* values.
+ *
+ * NOTE: at least {@link #testDefaultFsParameterLoading()} should not be run in parallel to other
+ * tests in the same JVM as it modifies a static (private) member of the {@link FileSystem} class
+ * and verifies its content.
*/
+@NotThreadSafe
public class TaskManagerConfigurationTest {
+ @Rule
+ public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
@Test
public void testUsePreconfiguredNetworkInterface() throws Exception {
final String TEST_HOST_NAME = "testhostname";
@@ -141,18 +150,18 @@ public class TaskManagerConfigurationTest {
}
@Test
- public void testDefaultFsParameterLoading() {
- final File tmpDir = getTmpDir();
- final File confFile = new File(tmpDir, GlobalConfiguration.FLINK_CONF_FILENAME);
-
+ public void testDefaultFsParameterLoading() throws Exception {
try {
+ final File tmpDir = temporaryFolder.newFolder();
+ final File confFile = new File(tmpDir, GlobalConfiguration.FLINK_CONF_FILENAME);
+
final URI defaultFS = new URI("otherFS", null, "localhost", 1234, null, null, null);
final PrintWriter pw1 = new PrintWriter(confFile);
- pw1.println("fs.default-scheme: "+ defaultFS);
+ pw1.println("fs.default-scheme: " + defaultFS);
pw1.close();
- String[] args = new String[]{"--configDir:" + tmpDir};
+ String[] args = new String[] {"--configDir:" + tmpDir};
TaskManager.parseArgsAndLoadConfig(args);
Field f = FileSystem.class.getDeclaredField("defaultScheme");
@@ -160,11 +169,11 @@ public class TaskManagerConfigurationTest {
URI scheme = (URI) f.get(null);
assertEquals("Default Filesystem Scheme not configured.", scheme, defaultFS);
- } catch (Exception e) {
- fail(e.getMessage());
} finally {
- confFile.delete();
- tmpDir.delete();
+ // reset default FS scheme:
+ Field f = FileSystem.class.getDeclaredField("defaultScheme");
+ f.setAccessible(true);
+ f.set(null, null);
}
}
@@ -210,10 +219,4 @@ public class TaskManagerConfigurationTest {
}
}
}
-
- private File getTmpDir() {
- File tmpDir = new File(CommonTestUtils.getTempDir(), UUID.randomUUID().toString());
- assertTrue("could not create temp directory", tmpDir.mkdirs());
- return tmpDir;
- }
}
[04/12] flink git commit: [FLINK-7249] [build] Bump java.version
property to 1.8
Posted by ch...@apache.org.
[FLINK-7249] [build] Bump java.version property to 1.8
This closes #4398.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0e553e0e
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0e553e0e
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0e553e0e
Branch: refs/heads/master
Commit: 0e553e0e6df1b53b609a4498d805c10d4dc19563
Parents: ca5d8af
Author: zentol <ch...@apache.org>
Authored: Mon Jul 24 13:13:34 2017 +0200
Committer: zentol <ch...@apache.org>
Committed: Mon Jul 31 12:12:08 2017 +0200
----------------------------------------------------------------------
docs/quickstart/java_api_quickstart.md | 2 +-
docs/quickstart/scala_api_quickstart.md | 2 +-
docs/quickstart/setup_quickstart.md | 2 +-
docs/setup/building.md | 4 ++--
.../src/main/resources/archetype-resources/pom.xml | 4 ++--
.../src/main/resources/archetype-resources/pom.xml | 4 ++--
pom.xml | 4 ++--
7 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/docs/quickstart/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md
index 7384d91..c21e06e 100644
--- a/docs/quickstart/java_api_quickstart.md
+++ b/docs/quickstart/java_api_quickstart.md
@@ -31,7 +31,7 @@ Start working on your Flink Java program in a few simple steps.
## Requirements
-The only requirements are working __Maven 3.0.4__ (or higher) and __Java 7.x__ (or higher) installations.
+The only requirements are working __Maven 3.0.4__ (or higher) and __Java 8.x__ (or higher) installations.
## Create Project
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/docs/quickstart/scala_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md
index 8ee1e66..abf6021 100644
--- a/docs/quickstart/scala_api_quickstart.md
+++ b/docs/quickstart/scala_api_quickstart.md
@@ -119,7 +119,7 @@ Now you can import the project into Eclipse via `File -> Import... -> Existing P
### Requirements
-The only requirements are working __Maven 3.0.4__ (or higher) and __Java 7.x__ (or higher) installations.
+The only requirements are working __Maven 3.0.4__ (or higher) and __Java 8.x__ (or higher) installations.
### Create Project
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/docs/quickstart/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/setup_quickstart.md b/docs/quickstart/setup_quickstart.md
index 0ddbd4e..7800dec 100644
--- a/docs/quickstart/setup_quickstart.md
+++ b/docs/quickstart/setup_quickstart.md
@@ -30,7 +30,7 @@ Get a Flink example program up and running in a few simple steps.
## Setup: Download and Start Flink
-Flink runs on __Linux, Mac OS X, and Windows__. To be able to run Flink, the only requirement is to have a working __Java 7.x__ (or higher) installation. Windows users, please take a look at the [Flink on Windows]({{ site.baseurl }}/setup/flink_on_windows.html) guide which describes how to run Flink on Windows for local setups.
+Flink runs on __Linux, Mac OS X, and Windows__. To be able to run Flink, the only requirement is to have a working __Java 8.x__ (or higher) installation. Windows users, please take a look at the [Flink on Windows]({{ site.baseurl }}/setup/flink_on_windows.html) guide which describes how to run Flink on Windows for local setups.
You can check the correct installation of Java by issuing the following command:
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/docs/setup/building.md
----------------------------------------------------------------------
diff --git a/docs/setup/building.md b/docs/setup/building.md
index 3ef372e..01dc4ed 100644
--- a/docs/setup/building.md
+++ b/docs/setup/building.md
@@ -31,10 +31,10 @@ This page covers how to build Flink {{ site.version }} from sources.
In order to build Flink you need the source code. Either [download the source of a release]({{ site.download_url }}) or [clone the git repository]({{ site.github_url }}).
-In addition you need **Maven 3** and a **JDK** (Java Development Kit). Flink requires **at least Java 7** to build. We recommend using Java 8.
+In addition you need **Maven 3** and a **JDK** (Java Development Kit). Flink requires **at least Java 8** to build.
*NOTE: Maven 3.3.x can build Flink, but will not properly shade away certain dependencies. Maven 3.0.3 creates the libraries properly.
-To build unit tests with Java 8, use Java 8u51 or above to prevent failures in unit tests that use the PowerMock runner.*
+To build unit tests use Java 8u51 or above to prevent failures in unit tests that use the PowerMock runner.*
To clone from git, enter:
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index a5f3963..5da38cd 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -307,8 +307,8 @@ under the License.
<artifactId>maven-compiler-plugin</artifactId>
<version>3.1</version>
<configuration>
- <source>1.7</source> <!-- If you want to use Java 8, change this to "1.8" -->
- <target>1.7</target> <!-- If you want to use Java 8, change this to "1.8" -->
+ <source>1.8</source>
+ <target>1.8</target>
</configuration>
</plugin>
</plugins>
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index e648851..67fe4c1 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -307,8 +307,8 @@ under the License.
<artifactId>maven-compiler-plugin</artifactId>
<version>3.1</version>
<configuration>
- <source>1.7</source>
- <target>1.7</target>
+ <source>1.8</source>
+ <target>1.8</target>
</configuration>
</plugin>
<plugin>
http://git-wip-us.apache.org/repos/asf/flink/blob/0e553e0e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 93d0470..3287f61 100644
--- a/pom.xml
+++ b/pom.xml
@@ -95,7 +95,7 @@ under the License.
<log4j.configuration>log4j-test.properties</log4j.configuration>
<guava.version>18.0</guava.version>
<akka.version>2.3-custom</akka.version>
- <java.version>1.7</java.version>
+ <java.version>1.8</java.version>
<slf4j.version>1.7.7</slf4j.version>
<log4j.version>1.2.17</log4j.version>
<!-- Overwrite default values from parent pom.
@@ -750,7 +750,7 @@ under the License.
<profile>
<id>release</id>
<properties>
- <java.version>1.7</java.version>
+ <java.version>1.8</java.version>
</properties>
<build>
<plugins>