From f457319481701a1234c9ea7d29da24f857ae4241 Mon Sep 17 00:00:00 2001 From: Maximilian Bode Date: Wed, 27 Apr 2016 17:19:16 +0200 Subject: [PATCH 01/70] [Flink-3836] Add LongHistogram accumulator --- docs/apis/common/index.md | 3 +- .../api/common/accumulators/Histogram.java | 42 +++++++--- .../common/accumulators/LongHistogram.java | 83 +++++++++++++++++++ .../api/common/functions/RuntimeContext.java | 7 ++ .../util/AbstractRuntimeUDFContext.java | 6 ++ .../common/accumulators/HistogramTest.java | 37 +++++++++ .../accumulators/LongHistogramTest.java | 45 ++++++++++ .../kafka/testutils/MockRuntimeContext.java | 6 ++ .../test/accumulators/AccumulatorITCase.java | 10 ++- 9 files changed, 223 insertions(+), 16 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongHistogram.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/accumulators/LongHistogramTest.java diff --git a/docs/apis/common/index.md b/docs/apis/common/index.md index cb2c78de7389d..049c5b39a6a19 100644 --- a/docs/apis/common/index.md +++ b/docs/apis/common/index.md @@ -1076,7 +1076,7 @@ interface. - {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java "__Histogram__" %}: A histogram implementation for a discrete number of bins. Internally it is just a map from Integer to Integer. You can use this to compute distributions of values, e.g. the distribution of - words-per-line for a word count program. + words-per-line for a word count program. (Alternatively use the {% gh_link /flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongHistogram.java "__LongHistogram__" %} if needed.) __How to use accumulators:__ @@ -1350,4 +1350,3 @@ You may specify program arguments before the job is executed. The plan visualiza the execution plan before executing the Flink job. {% top %} - diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java index 577795b0d81c0..0c7370baad99e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common.accumulators; import org.apache.flink.annotation.Public; +import org.apache.flink.util.MathUtils; import java.util.Map; import java.util.TreeMap; @@ -27,7 +28,7 @@ * Histogram accumulator, which builds a histogram in a distributed manner. * Implemented as a Integer->Integer TreeMap, so that the entries are sorted * according to the values. - * + * * This class does not extend to continuous values later, because it makes no * attempt to put the data in bins. */ @@ -36,47 +37,62 @@ public class Histogram implements Accumulator private static final long serialVersionUID = 1L; - private TreeMap treeMap = new TreeMap(); + private LongHistogram internalHistogram = new LongHistogram(); @Override public void add(Integer value) { - Integer current = treeMap.get(value); - Integer newValue = (current != null ? current : 0) + 1; - this.treeMap.put(value, newValue); + internalHistogram.add(value); } @Override public TreeMap getLocalValue() { - return this.treeMap; + final TreeMap longTreeMap = internalHistogram.getLocalValue(); + return convertToIntMap(longTreeMap); + } + + private TreeMap convertToIntMap(final TreeMap longTreeMap) { + final TreeMap intTreeMap = new TreeMap<>(); + for (final Map.Entry entry : longTreeMap.entrySet()) { + intTreeMap.put(entry.getKey(), checkedCast(entry.getValue())); + } + return intTreeMap; + } + + private int checkedCast(final Long l) { + try { + return MathUtils.checkedDownCast(l); + } catch (final IllegalArgumentException e) { + throw new IllegalArgumentException("Histogram can only deal with int values, consider using LongHistogram.", e); + } } @Override public void merge(Accumulator> other) { // Merge the values into this map for (Map.Entry entryFromOther : other.getLocalValue().entrySet()) { - Integer ownValue = this.treeMap.get(entryFromOther.getKey()); + Long ownValue = internalHistogram.getLocalValue().get(entryFromOther.getKey()); if (ownValue == null) { - this.treeMap.put(entryFromOther.getKey(), entryFromOther.getValue()); + internalHistogram.getLocalValue().put(entryFromOther.getKey(), entryFromOther.getValue().longValue()); } else { - this.treeMap.put(entryFromOther.getKey(), entryFromOther.getValue() + ownValue); + internalHistogram.getLocalValue().put(entryFromOther.getKey(), entryFromOther.getValue() + ownValue); } } } @Override public void resetLocal() { - this.treeMap.clear(); + internalHistogram.resetLocal(); } @Override public String toString() { - return this.treeMap.toString(); + return internalHistogram.toString(); } @Override public Accumulator> clone() { - Histogram result = new Histogram(); - result.treeMap = new TreeMap(treeMap); + final Histogram result = new Histogram(); + result.internalHistogram = internalHistogram; return result; } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongHistogram.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongHistogram.java new file mode 100644 index 0000000000000..46201ec6b5643 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/LongHistogram.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.accumulators; + +import org.apache.flink.annotation.Public; + +import java.util.Map; +import java.util.TreeMap; + +/** + * Histogram accumulator, which builds a histogram in a distributed manner. + * Basically the same as {@link Histogram} with Long instead of Integer values. + * Implemented as a Integer->Long TreeMap, so that the entries are sorted + * according to the values. + * + * This class does not extend to continuous values later, because it makes no + * attempt to put the data in bins. + */ +@Public +public class LongHistogram implements Accumulator> { + + private static final long serialVersionUID = 1L; + + private TreeMap treeMap = new TreeMap(); + + @Override + public void add(Integer value) { + Long current = treeMap.get(value); + Long newValue = (current != null ? current : 0) + 1; + this.treeMap.put(value, newValue); + } + + @Override + public TreeMap getLocalValue() { + return this.treeMap; + } + + @Override + public void merge(Accumulator> other) { + // Merge the values into this map + for (Map.Entry entryFromOther : other.getLocalValue().entrySet()) { + Long ownValue = this.treeMap.get(entryFromOther.getKey()); + if (ownValue == null) { + this.treeMap.put(entryFromOther.getKey(), entryFromOther.getValue()); + } else { + this.treeMap.put(entryFromOther.getKey(), entryFromOther.getValue() + ownValue); + } + } + } + + @Override + public void resetLocal() { + this.treeMap.clear(); + } + + @Override + public String toString() { + return this.treeMap.toString(); + } + + @Override + public Accumulator> clone() { + LongHistogram result = new LongHistogram(); + result.treeMap = new TreeMap(treeMap); + return result; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 3225c003550d4..a23964e572ef7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.accumulators.Histogram; import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.accumulators.LongHistogram; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -151,6 +152,12 @@ public interface RuntimeContext { */ @PublicEvolving Histogram getHistogram(String name); + + /** + * Convenience function to create a counter object for long histograms. + */ + @PublicEvolving + LongHistogram getLongHistogram(String name); // -------------------------------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java index 74b78df624ee9..9b139c61e5569 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java @@ -32,6 +32,7 @@ import org.apache.flink.api.common.accumulators.Histogram; import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.accumulators.LongHistogram; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.ListState; @@ -119,6 +120,11 @@ public Histogram getHistogram(String name) { return (Histogram) getAccumulator(name, Histogram.class); } + @Override + public LongHistogram getLongHistogram(String name) { + return (LongHistogram) getAccumulator(name, LongHistogram.class); + } + @Override public DoubleCounter getDoubleCounter(String name) { return (DoubleCounter) getAccumulator(name, DoubleCounter.class); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java b/flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java new file mode 100644 index 0000000000000..f18cc2332e51a --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.accumulators; + +import org.junit.Test; + +public class HistogramTest { + + private static final Integer KEY = 1; + public static final long MORE_THAN_AN_INT = (long) Integer.MAX_VALUE + 1; + + @Test(expected = IllegalArgumentException.class) + public void longValueThrowsException() { + Histogram histogram = new Histogram(); + for (long i = 0; i < MORE_THAN_AN_INT; i++) { + histogram.add(KEY); + } + + histogram.getLocalValue(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/accumulators/LongHistogramTest.java b/flink-core/src/test/java/org/apache/flink/api/common/accumulators/LongHistogramTest.java new file mode 100644 index 0000000000000..6f5998ba2fb56 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/accumulators/LongHistogramTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.accumulators; + +import org.junit.Test; +import java.util.TreeMap; + +import static org.junit.Assert.assertThat; +import static org.hamcrest.CoreMatchers.equalTo; + +public class LongHistogramTest { + + private static final Integer KEY = 1; + public static final long MORE_THAN_AN_INT = (long) Integer.MAX_VALUE + 1; + + @Test + public void canDealWithLongValue() { + LongHistogram histogram = new LongHistogram(); + for (long i = 0; i < MORE_THAN_AN_INT; i++) { + histogram.add(KEY); + } + + TreeMap expectedMap = new TreeMap<>(); + expectedMap.put(KEY, MORE_THAN_AN_INT); + + assertThat(histogram.getLocalValue(), equalTo(expectedMap)); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java index 1ac2ef53637c2..ea301f3eb8dbe 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.accumulators.Histogram; import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.accumulators.LongHistogram; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.state.ListState; @@ -147,6 +148,11 @@ public Histogram getHistogram(String name) { throw new UnsupportedOperationException(); } + @Override + public LongHistogram getLongHistogram(String name) { + throw new UnsupportedOperationException(); + } + @Override public List getBroadcastVariable(String name) { throw new UnsupportedOperationException(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java index b4015e5994c34..a2f4eca587346 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.accumulators.DoubleCounter; import org.apache.flink.api.common.accumulators.Histogram; import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.accumulators.LongHistogram; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; @@ -84,7 +85,11 @@ protected void postSubmit() throws Exception { Map dist = Maps.newHashMap(); dist.put(1, 1); dist.put(2, 1); dist.put(3, 1); Assert.assertEquals(dist, res.getAccumulatorResult("words-per-line")); - + + Map distLong = Maps.newHashMap(); + distLong.put(1, 1L); distLong.put(2, 1L); distLong.put(3, 1L); + Assert.assertEquals(distLong, res.getAccumulatorResult("words-per-line-long")); + // Test distinct words (custom accumulator) Set distinctWords = Sets.newHashSet(); distinctWords.add(new StringValue("one")); @@ -113,6 +118,7 @@ public static class TokenizeLine extends RichFlatMapFunction> out) { ++ wordsPerLine; } wordsPerLineDistribution.add(wordsPerLine); + wordsPerLineDistributionLong.add(wordsPerLine); } @Override From 7d6dfdfef9b09fe3a339cc3a2da61f31fde6a3aa Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Fri, 6 May 2016 14:46:02 +0200 Subject: [PATCH 02/70] [docs] Add note about S3AFileSystem 'buffer.dir' property --- docs/setup/aws.md | 30 +++++++++++++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/docs/setup/aws.md b/docs/setup/aws.md index 156b4f0c068d8..ce1dcfa1ad9bf 100644 --- a/docs/setup/aws.md +++ b/docs/setup/aws.md @@ -95,13 +95,20 @@ There are two popular S3 file system implementations available: This is the recommended S3 FileSystem implementation to use. It uses Amazon's SDK internally and works with IAM roles (see [Configure Access Credential](#configure-access-credentials)). -You need to point Flink to a valid Hadoop configuration, which contains the following property in `core-site.xml`: +You need to point Flink to a valid Hadoop configuration, which contains the following properties in `core-site.xml`: ```xml fs.s3.impl org.apache.hadoop.fs.s3a.S3AFileSystem + + + + fs.s3.buffer.dir + /tmp + ``` This registers `S3AFileSystem` as the default FileSystem for URIs with the `s3://` scheme. @@ -311,3 +318,24 @@ Caused by: org.jets3t.service.impl.rest.HttpException [...] ``` {% top %} + +### NullPointerException at org.apache.hadoop.fs.LocalDirAllocator + +This Exception is usually caused by skipping the local buffer directory configuration `fs.s3.buffer.dir` for the `S3AFileSystem`. Please refer to the [S3AFileSystem configuration](#s3afilesystem-recommended) section to see how to configure the `S3AFileSystem` properly. + +``` +[...] +Caused by: java.lang.NullPointerException at +o.a.h.fs.LocalDirAllocator$AllocatorPerContext.confChanged(LocalDirAllocator.java:268) at +o.a.h.fs.LocalDirAllocator$AllocatorPerContext.getLocalPathForWrite(LocalDirAllocator.java:344) at +o.a.h.fs.LocalDirAllocator$AllocatorPerContext.createTmpFileForWrite(LocalDirAllocator.java:416) at +o.a.h.fs.LocalDirAllocator.createTmpFileForWrite(LocalDirAllocator.java:198) at +o.a.h.fs.s3a.S3AOutputStream.(S3AOutputStream.java:87) at +o.a.h.fs.s3a.S3AFileSystem.create(S3AFileSystem.java:410) at +o.a.h.fs.FileSystem.create(FileSystem.java:907) at +o.a.h.fs.FileSystem.create(FileSystem.java:888) at +o.a.h.fs.FileSystem.create(FileSystem.java:785) at +o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:404) at +o.a.f.runtime.fs.hdfs.HadoopFileSystem.create(HadoopFileSystem.java:48) at +... 25 more +``` From dce10b64c1aa5897a8e00a1bffcd07f66826c964 Mon Sep 17 00:00:00 2001 From: mans2singh Date: Sat, 7 May 2016 12:08:02 -0700 Subject: [PATCH 03/70] [FLINK-3881] [docs] Java 8 Documetation Sample Correction This closes #1970. --- docs/apis/java8.md | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/docs/apis/java8.md b/docs/apis/java8.md index 821038bf88e7b..c82035589c2be 100644 --- a/docs/apis/java8.md +++ b/docs/apis/java8.md @@ -53,29 +53,33 @@ If the `Collector` type can not be inferred from the surrounding context, it nee Otherwise the output will be treated as type `Object` which can lead to undesired behaviour. ~~~java -DataSet input = env.fromElements(1, 2, 3); +DataSet input = env.fromElements(1, 2, 3); // collector type must be declared input.flatMap((Integer number, Collector out) -> { + StringBuilder builder = new StringBuilder(); for(int i = 0; i < number; i++) { - out.collect("a"); + builder.append("a"); + out.collect(builder.toString()); } }) -// returns "a", "a", "aa", "a", "aa" , "aaa" +// returns (on separate lines) "a", "a", "aa", "a", "aa", "aaa" .print(); ~~~ ~~~java -DataSet input = env.fromElements(1, 2, 3); +DataSet input = env.fromElements(1, 2, 3); // collector type must not be declared, it is inferred from the type of the dataset DataSet manyALetters = input.flatMap((number, out) -> { + StringBuilder builder = new StringBuilder(); for(int i = 0; i < number; i++) { - out.collect("a"); + builder.append("a"); + out.collect(builder.toString()); } }); -// returns "a", "a", "aa", "a", "aa" , "aaa" +// returns (on separate lines) "a", "a", "aa", "a", "aa", "aaa" manyALetters.print(); ~~~ From ad3a70d3a9c6d54e7092575a87457fc0ed0e9d42 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Mon, 9 May 2016 10:59:31 +0200 Subject: [PATCH 04/70] [docs] Adjust network buffer config for slots and add tl;dr --- docs/setup/config.md | 44 +++++++++++++++++++++++++++----------------- 1 file changed, 27 insertions(+), 17 deletions(-) diff --git a/docs/setup/config.md b/docs/setup/config.md index 6e6e0a6fea8d2..db189a0674aaa 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -54,11 +54,11 @@ The configuration files for the TaskManagers can be different, Flink does not as - `parallelism.default`: The default parallelism to use for programs that have no parallelism specified. (DEFAULT: 1). For setups that have no concurrent jobs running, setting this value to NumTaskManagers * NumSlotsPerTaskManager will cause the system to use all available execution resources for the program's execution. **Note**: The default parallelism can be overwriten for an entire job by calling `setParallelism(int parallelism)` on the `ExecutionEnvironment` or by passing `-p ` to the Flink Command-line frontend. It can be overwritten for single transformations by calling `setParallelism(int parallelism)` on an operator. See the [programming guide]({{site.baseurl}}/apis/programming_guide.html#parallel-execution) for more information about the parallelism. -- `fs.default-scheme`: The default filesystem scheme to be used, with the necessary authority to contact, e.g. the host:port of the NameNode in the case of HDFS (if needed). -By default, this is set to `file:///` which points to the local filesystem. This means that the local -filesystem is going to be used to search for user-specified files **without** an explicit scheme -definition. As another example, if this is set to `hdfs://localhost:9000/`, then a user-specified file path -without explicit scheme definition, such as `/user/USERNAME/in.txt`, is going to be transformed into +- `fs.default-scheme`: The default filesystem scheme to be used, with the necessary authority to contact, e.g. the host:port of the NameNode in the case of HDFS (if needed). +By default, this is set to `file:///` which points to the local filesystem. This means that the local +filesystem is going to be used to search for user-specified files **without** an explicit scheme +definition. As another example, if this is set to `hdfs://localhost:9000/`, then a user-specified file path +without explicit scheme definition, such as `/user/USERNAME/in.txt`, is going to be transformed into `hdfs://localhost:9000/user/USERNAME/in.txt`. This scheme is used **ONLY** if no other scheme is specified (explicitly) in the user-provided `URI`. - `fs.hdfs.hadoopconf`: The absolute path to the Hadoop File System's (HDFS) configuration **directory** (OPTIONAL VALUE). Specifying this value allows programs to reference HDFS files using short URIs (`hdfs:///path/to/files`, without including the address and port of the NameNode in the file URI). Without this option, HDFS files can be accessed, but require fully qualified URIs like `hdfs://address:port/path/to/files`. This option also causes file writers to pick up the HDFS's default values for block sizes and replication factors. Flink will look for the "core-site.xml" and "hdfs-site.xml" files in the specified directory. @@ -133,11 +133,11 @@ To use the fixed delay strategy you have to specify "fixed-delay". To turn the restart behaviour off you have to specify "none". Default value "none". -- `restart-strategy.fixed-delay.attempts`: Number of restart attempts, used if the default restart strategy is set to "fixed-delay". +- `restart-strategy.fixed-delay.attempts`: Number of restart attempts, used if the default restart strategy is set to "fixed-delay". Default value is 1. - -- `restart-strategy.fixed-delay.delay`: Delay between restart attempts, used if the default restart strategy is set to "fixed-delay". -Default value is the `akka.ask.timeout`. + +- `restart-strategy.fixed-delay.delay`: Delay between restart attempts, used if the default restart strategy is set to "fixed-delay". +Default value is the `akka.ask.timeout`. ## Full Reference @@ -204,9 +204,9 @@ The following parameters configure Flink's JobManager and TaskManagers. The parameters define the behavior of tasks that create result files. -- `fs.default-scheme`: The default filesystem scheme to be used, with the necessary authority to contact, e.g. the host:port of the NameNode in the case of HDFS (if needed). -By default, this is set to `file:///` which points to the local filesystem. This means that the local -filesystem is going to be used to search for user-specified files **without** an explicit scheme +- `fs.default-scheme`: The default filesystem scheme to be used, with the necessary authority to contact, e.g. the host:port of the NameNode in the case of HDFS (if needed). +By default, this is set to `file:///` which points to the local filesystem. This means that the local +filesystem is going to be used to search for user-specified files **without** an explicit scheme definition. This scheme is used **ONLY** if no other scheme is specified (explicitly) in the user-provided `URI`. - `fs.overwrite-files`: Specifies whether file output writers should overwrite existing files by default. Set to *true* to overwrite by default, *false* otherwise. (DEFAULT: false) @@ -240,10 +240,10 @@ definition. This scheme is used **ONLY** if no other scheme is specified (explic - `yarn.properties-file.location` (Default: temp directory). When a Flink job is submitted to YARN, the JobManager's host and the number of available processing slots is written into a properties file, so that the Flink clientis able to pick those details up. This configuration parameter allows changing the default location of that file (for example for environments sharing a Flink installation between users) - `yarn.application-master.env.`*ENV_VAR1=value* Configuration values prefixed with `yarn.application-master.env.` will be passed as environment variables to the ApplicationMaster/JobManager process. For example for passing `LD_LIBRARY_PATH` as an env variable to the ApplicationMaster, set: - + yarn.application-master.env.LD_LIBRARY_PATH: "/usr/lib/native" -- `yarn.containers.vcores` The number of virtual cores (vcores) per YARN container. By default, the number of `vcores` is set to the number of slots per TaskManager, if set, or to 1, otherwise. +- `yarn.containers.vcores` The number of virtual cores (vcores) per YARN container. By default, the number of `vcores` is set to the number of slots per TaskManager, if set, or to 1, otherwise. - `yarn.taskmanager.env.` Similar to the configuration prefix about, this prefix allows setting custom environment variables for the TaskManager processes. @@ -274,7 +274,7 @@ For example when running Flink on YARN on an environment with a restrictive fire - `recovery.zookeeper.client.max-retry-attempts`: (Default '3') Defines the number of connection retries before the client gives up. -- `recovery.job.delay`: (Default 'akka.ask.timeout') Defines the delay before persisted jobs are recovered in case of a recovery situation. +- `recovery.job.delay`: (Default 'akka.ask.timeout') Defines the delay before persisted jobs are recovered in case of a recovery situation. ## Environment @@ -284,12 +284,22 @@ For example when running Flink on YARN on an environment with a restrictive fire ### Configuring the Network Buffers +If you ever see the Exception `java.io.IOException: Insufficient number of network buffers`, please use the following formula to adjust the number of network buffers: + +``` +#slots-per-TM^2 * #TMs * 4 +``` + +Where `#slots per TM` are the [number of slots per TaskManager](#configuring-taskmanager-processing-slots) and `#TMs` are the total number of task managers. + Network buffers are a critical resource for the communication layers. They are used to buffer records before transmission over a network, and to buffer incoming data before dissecting it into records and handing them to the application. A sufficient number of network buffers is critical to achieve a good throughput. -In general, configure the task manager to have enough buffers that each logical network connection on you expect to be open at the same time has a dedicated buffer. A logical network connection exists for each point-to-point exchange of data over the network, which typically happens at repartitioning- or broadcasting steps. In those, each parallel task inside the TaskManager has to be able to talk to all other parallel tasks. Hence, the required number of buffers on a task manager is *total-degree-of-parallelism* (number of targets) \* *intra-node-parallelism* (number of sources in one task manager) \* *n*. Here, *n* is a constant that defines how many repartitioning-/broadcasting steps you expect to be active at the same time. +In general, configure the task manager to have enough buffers that each logical network connection you expect to be open at the same time has a dedicated buffer. A logical network connection exists for each point-to-point exchange of data over the network, which typically happens at repartitioning- or broadcasting steps (shuffle phase). In those, each parallel task inside the TaskManager has to be able to talk to all other parallel tasks. Hence, the required number of buffers on a task manager is *total-degree-of-parallelism* (number of targets) \* *intra-node-parallelism* (number of sources in one task manager) \* *n*. Here, *n* is a constant that defines how many repartitioning-/broadcasting steps you expect to be active at the same time. + +Since the *intra-node-parallelism* is typically the number of cores, and more than 4 repartitioning or broadcasting channels are rarely active in parallel, it frequently boils down to `#slots-per-TM^2 * #TMs * 4`. -Since the *intra-node-parallelism* is typically the number of cores, and more than 4 repartitioning or broadcasting channels are rarely active in parallel, it frequently boils down to *\#cores\^2\^* \* *\#machines* \* 4. To support for example a cluster of 20 8-core machines, you should use roughly 5000 network buffers for optimal throughput. +To support for example a cluster of 20 8-slot machines, you should use roughly 5000 network buffers for optimal throughput. Each network buffer has by default a size of 32 KiBytes. In the above example, the system would allocate roughly 300 MiBytes for network buffers. From d3f11a1710401c3ce6343e88659d3cfcfeb92348 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Sat, 16 Apr 2016 10:30:58 -0400 Subject: [PATCH 05/70] [FLINK-3772] [gelly] Graph algorithms for vertex and edge degree Graph algorithms for annotating vertex degree for undirected graphs vertex out-, in-, and out- and in-degree for directed graphs edge source, target, and source and target degree for undirected graphs --- docs/apis/batch/libs/gelly.md | 130 +++++++++- .../annotate/DegreeAnnotationFunctions.java | 236 ++++++++++++++++++ .../annotate/directed/VertexDegreePair.java | 107 ++++++++ .../annotate/directed/VertexInDegree.java | 103 ++++++++ .../annotate/directed/VertexOutDegree.java | 103 ++++++++ .../asm/degree/annotate/package-info.java | 41 +++ .../annotate/undirected/EdgeDegreePair.java | 108 ++++++++ .../annotate/undirected/EdgeSourceDegree.java | 101 ++++++++ .../annotate/undirected/EdgeTargetDegree.java | 101 ++++++++ .../annotate/undirected/VertexDegree.java | 126 ++++++++++ .../apache/flink/graph/asm/AsmTestBase.java | 102 ++++++++ .../directed/VertexDegreePairTest.java | 87 +++++++ .../annotate/directed/VertexInDegreeTest.java | 87 +++++++ .../directed/VertexOutDegreeTest.java | 86 +++++++ .../undirected/EdgeDegreePairTest.java | 84 +++++++ .../undirected/EdgeSourceDegreeTest.java | 84 +++++++ .../undirected/EdgeTargetDegreeTest.java | 84 +++++++ .../annotate/undirected/VertexDegreeTest.java | 119 +++++++++ 18 files changed, 1886 insertions(+), 3 deletions(-) create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/DegreeAnnotationFunctions.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePair.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/package-info.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePairTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java diff --git a/docs/apis/batch/libs/gelly.md b/docs/apis/batch/libs/gelly.md index fb9bc433ebfee..79aa68edf9a46 100644 --- a/docs/apis/batch/libs/gelly.md +++ b/docs/apis/batch/libs/gelly.md @@ -121,6 +121,9 @@ val weight = e.getValue // weight = 0.5 +In Gelly an `Edge` is always directed from the source vertex to the target vertex. A `Graph` may be undirected if for +every `Edge` it contains a matching `Edge` from the target vertex to the source vertex. + {% top %} Graph Creation @@ -2067,7 +2070,128 @@ configuration. - TranslateGraphIds + degree.annotate.directed.
VertexInDegree + +

Annotate vertices of a directed graph with the in-degree.

+{% highlight java %} +DataSet> inDegree = graph + .run(new VertexInDegree() + .setIncludeZeroDegreeVertices(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setIncludeZeroDegreeVertices: by default only the edge set is processed for the computation of degree; when this flag is set an additional join is performed against the vertex set in order to output vertices with an in-degree of zero

  • +
  • setParallelism: override the operator parallelism

  • +
+ + + + + degree.annotate.directed.
VertexOutDegree + +

Annotate vertices of a directed graph with the out-degree.

+{% highlight java %} +DataSet> outDegree = graph + .run(new VertexOutDegree() + .setIncludeZeroDegreeVertices(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setIncludeZeroDegreeVertices: by default only the edge set is processed for the computation of degree; when this flag is set an additional join is performed against the vertex set in order to output vertices with an out-degree of zero

  • +
  • setParallelism: override the operator parallelism

  • +
+ + + + + degree.annotate.directed.
VertexDegreePair + +

Annotate vertices of a directed graph with both the out-degree and in-degree.

+{% highlight java %} +DataSet>> pairDegree = graph + .run(new VertexDegreePair() + .setIncludeZeroDegreeVertices(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setIncludeZeroDegreeVertices: by default only the edge set is processed for the computation of degree; when this flag is set an additional join is performed against the vertex set in order to output vertices with out- and in-degree of zero

  • +
  • setParallelism: override the operator parallelism

  • +
+ + + + + degree.annotate.undirected.
VertexDegree + +

Annotate vertices of an undirected graph with the degree.

+{% highlight java %} +DataSet> degree = graph + .run(new VertexDegree() + .setIncludeZeroDegreeVertices(true) + .setReduceOnTargetId(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setIncludeZeroDegreeVertices: by default only the edge set is processed for the computation of degree; when this flag is set an additional join is performed against the vertex set in order to output vertices with a degree of zero

  • +
  • setParallelism: override the operator parallelism

  • +
  • setReduceOnTargetId: the degree can be counted from either the edge source or target IDs. By default the source IDs are counted. Reducing on target IDs may optimize the algorithm if the input edge list is sorted by target ID.

  • +
+ + + + + degree.annotate.undirected.
EdgeSourceDegree + +

Annotate edges of an undirected graph with degree of the source ID.

+{% highlight java %} +DataSet>> sourceDegree = graph + .run(new EdgeSourceDegree() + .setReduceOnTargetId(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setParallelism: override the operator parallelism

  • +
  • setReduceOnTargetId: the degree can be counted from either the edge source or target IDs. By default the source IDs are counted. Reducing on target IDs may optimize the algorithm if the input edge list is sorted by target ID.

  • +
+ + + + + degree.annotate.undirected.
EdgeTargetDegree + +

Annotate edges of an undirected graph with degree of the target ID.

+{% highlight java %} +DataSet>> targetDegree = graph + .run(new EdgeTargetDegree() + .setReduceOnSourceId(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setParallelism: override the operator parallelism

  • +
  • setReduceOnSourceId: the degree can be counted from either the edge source or target IDs. By default the target IDs are counted. Reducing on source IDs may optimize the algorithm if the input edge list is sorted by source ID.

  • +
+ + + + + degree.annotate.undirected.
EdgeDegreePair + +

Annotate edges of an undirected graph with the degree of both the source and target degree ID.

+{% highlight java %} +DataSet>> pairDegree = graph + .run(new EdgeDegreePair() + .setReduceOnTargetId(true)); +{% endhighlight %} +

Optional configuration:

+
    +
  • setParallelism: override the operator parallelism

  • +
  • setReduceOnTargetId: the degree can be counted from either the edge source or target IDs. By default the source IDs are counted. Reducing on target IDs may optimize the algorithm if the input edge list is sorted by target ID.

  • +
+ + + + + translate.
TranslateGraphIds

Translate vertex and edge IDs using the given MapFunction.

{% highlight java %} @@ -2077,7 +2201,7 @@ graph.run(new TranslateGraphIds(new LongValueToStringValue())); - TranslateVertexValues + translate.
TranslateVertexValues

Translate vertex values using the given MapFunction.

{% highlight java %} @@ -2087,7 +2211,7 @@ graph.run(new TranslateVertexValues(new LongValueAddOffset(vertexCount))); - TranslateEdgeValues + translate.
TranslateEdgeValues

Translate edge values using the given MapFunction.

{% highlight java %} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/DegreeAnnotationFunctions.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/DegreeAnnotationFunctions.java new file mode 100644 index 0000000000000..098e9fe05facd --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/DegreeAnnotationFunctions.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate; + +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Vertex; +import org.apache.flink.types.LongValue; + +public class DegreeAnnotationFunctions { + + // -------------------------------------------------------------------------------------------- + // Vertex functions + // -------------------------------------------------------------------------------------------- + + /** + * Emits the source vertex ID along with an initial count. + * + * @param ID type + * @param edge value type + */ + @ForwardedFields("0") + public static class MapEdgeToSourceId + implements MapFunction, Vertex> { + private Vertex output = new Vertex<>(null, new LongValue(1)); + + @Override + public Vertex map(Edge value) throws Exception { + output.f0 = value.f0; + return output; + } + } + + /** + * Emits the target vertex ID along with an initial count. + * + * @param ID type + * @param edge value type + */ + @ForwardedFields("1->0") + public static class MapEdgeToTargetId + implements MapFunction, Vertex> { + private Vertex output = new Vertex<>(null, new LongValue(1)); + + @Override + public Vertex map(Edge value) throws Exception { + output.f0 = value.f1; + return output; + } + } + + /** + * Combines the vertex degree count. + * + * @param ID type + */ + @ForwardedFields("0") + public static class DegreeCount + implements ReduceFunction> { + @Override + public Vertex reduce(Vertex left, Vertex right) + throws Exception { + LongValue count = left.f1; + count.setValue(count.getValue() + right.f1.getValue()); + return left; + } + } + + /** + * Performs a left outer join to apply a zero count for vertices with + * out- or in-degree of zero. + * + * @param ID type + * @param vertex value type + */ + @ForwardedFieldsFirst("0") + @ForwardedFieldsSecond("0") + public static final class JoinVertexWithVertexDegree + implements JoinFunction, Vertex, Vertex> { + private LongValue zero = new LongValue(0); + + private Vertex output = new Vertex<>(); + + @Override + public Vertex join(Vertex vertex, Vertex vertexDegree) + throws Exception { + output.f0 = vertex.f0; + output.f1 = (vertexDegree == null) ? zero : vertexDegree.f1; + + return output; + } + } + + /** + * Performs a left outer join to apply a zero count for vertices with + * out- and in-degree of zero. + * + * @param ID type + * @param vertex value type + */ + @ForwardedFieldsFirst("0") + @ForwardedFieldsSecond("0") + public static final class JoinVertexWithVertexDegrees + implements JoinFunction, Vertex>, Vertex>> { + private Tuple2 zeros; + + private Vertex> output = new Vertex<>(); + + public JoinVertexWithVertexDegrees() { + LongValue zero = new LongValue(0); + zeros = new Tuple2<>(zero, zero); + } + @Override + public Vertex> join(Vertex vertex, Vertex> vertexDegree) + throws Exception { + output.f0 = vertex.f0; + output.f1 = (vertexDegree == null) ? zeros : vertexDegree.f1; + + return output; + } + } + + /** + * Performs a full outer join composing vertex out- and in-degree and + * applying a zero count for vertices having an out- or in-degree of zero. + * + * @param ID type + */ + @ForwardedFieldsFirst("0") + @ForwardedFieldsSecond("0") + public static final class JoinVertexDegreeWithVertexDegree + implements JoinFunction, Vertex, Vertex>> { + private LongValue zero = new LongValue(0); + + private Tuple2 degrees = new Tuple2<>(); + + private Vertex> output = new Vertex<>(null, degrees); + + @Override + public Vertex> join(Vertex left, Vertex right) + throws Exception { + if (left == null) { + output.f0 = right.f0; + degrees.f0 = zero; + degrees.f1 = right.f1; + } else { + output.f0 = left.f0; + degrees.f0 = left.f1; + degrees.f1 = (right == null) ? zero : right.f1; + } + + return output; + } + } + + // -------------------------------------------------------------------------------------------- + // Edge functions + // -------------------------------------------------------------------------------------------- + + /** + * Assigns the vertex degree to this edge value. + * + * @param ID type + * @param edge value type + */ + @ForwardedFieldsFirst("0; 1; 2->2.0") + @ForwardedFieldsSecond("0; 1->2.1") + public static final class JoinEdgeWithVertexDegree + implements JoinFunction, Vertex, Edge>> { + private Tuple2 valueAndDegree = new Tuple2<>(); + + private Edge> output = new Edge<>(null, null, valueAndDegree); + + @Override + public Edge> join(Edge edge, Vertex vertex) throws Exception { + output.f0 = edge.f0; + output.f1 = edge.f1; + valueAndDegree.f0 = edge.f2; + valueAndDegree.f1 = vertex.f1; + + return output; + } + } + + /** + * Composes the vertex degree with this edge value. + * + * @param ID type + * @param edge value type + */ + @ForwardedFieldsFirst("0; 1; 2.0; 2.1") + @ForwardedFieldsSecond("0; 1->2.2") + public static final class JoinEdgeDegreeWithVertexDegree + implements JoinFunction>, Vertex, Edge>> { + private Tuple3 valueAndDegrees = new Tuple3<>(); + + private Edge> output = new Edge<>(null, null, valueAndDegrees); + + @Override + public Edge> join(Edge> edge, Vertex vertex) + throws Exception { + Tuple2 valueAndDegree = edge.f2; + + output.f0 = edge.f0; + output.f1 = edge.f1; + valueAndDegrees.f0 = valueAndDegree.f0; + valueAndDegrees.f1 = valueAndDegree.f1; + valueAndDegrees.f2 = vertex.f1; + + return output; + } + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePair.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePair.java new file mode 100644 index 0000000000000..3e2cae0bcef66 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePair.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.directed; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinVertexDegreeWithVertexDegree; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinVertexWithVertexDegrees; +import org.apache.flink.types.LongValue; + +/** + * Annotates vertices of a directed graph with both the out-degree and in-degree. + * + * @param ID type + * @param vertex value type + * @param edge value type + */ +public class VertexDegreePair +implements GraphAlgorithm>>> { + + // Optional configuration + private boolean includeZeroDegreeVertices = false; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * By default only the edge set is processed for the computation of degree. + * When this flag is set an additional join is performed against the vertex + * set in order to output vertices with out- and in-degree of zero. + * + * @param includeZeroDegreeVertices whether to output vertices with out- + * and in-degree of zero + * @return this + */ + public VertexDegreePair setIncludeZeroDegreeVertices(boolean includeZeroDegreeVertices) { + this.includeZeroDegreeVertices = includeZeroDegreeVertices; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public VertexDegreePair setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet>> run(Graph input) + throws Exception { + // s, deg(s) + DataSet> outDegree = input + .run(new VertexOutDegree() + .setIncludeZeroDegreeVertices(false)); + + // t, deg(t) + DataSet> inDegree = input + .run(new VertexInDegree() + .setIncludeZeroDegreeVertices(false)); + + DataSet>> degree = outDegree + .fullOuterJoin(inDegree) + .where(0) + .equalTo(0) + .with(new JoinVertexDegreeWithVertexDegree()) + .setParallelism(parallelism) + .name("Join out- and in-degree"); + + if (includeZeroDegreeVertices) { + degree = input + .getVertices() + .leftOuterJoin(degree) + .where(0) + .equalTo(0) + .with(new JoinVertexWithVertexDegrees()) + .setParallelism(parallelism) + .name("Join zero degree vertices"); + } + + return degree; + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java new file mode 100644 index 0000000000000..5825628593d5a --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegree.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.directed; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.DegreeCount; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinVertexWithVertexDegree; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.MapEdgeToTargetId; +import org.apache.flink.types.LongValue; + +/** + * Annotates vertices of a directed graph with the in-degree. + * + * @param graph label type + * @param vertex value type + * @param edge value type + */ +public class VertexInDegree +implements GraphAlgorithm>> { + + // Optional configuration + private boolean includeZeroDegreeVertices = true; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * By default only the edge set is processed for the computation of degree. + * When this flag is set an additional join is performed against the vertex + * set in order to output vertices with an in-degree of zero. + * + * @param includeZeroDegreeVertices whether to output vertices with an + * in-degree of zero + * @return this + */ + public VertexInDegree setIncludeZeroDegreeVertices(boolean includeZeroDegreeVertices) { + this.includeZeroDegreeVertices = includeZeroDegreeVertices; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public VertexInDegree setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet> run(Graph input) + throws Exception { + // t + DataSet> targetIds = input + .getEdges() + .map(new MapEdgeToTargetId()) + .setParallelism(parallelism) + .name("Map edge to target ID"); + + // t, deg(t) + DataSet> targetDegree = targetIds + .groupBy(0) + .reduce(new DegreeCount()) + .setParallelism(parallelism) + .name("Degree count"); + + if (includeZeroDegreeVertices) { + targetDegree = input + .getVertices() + .leftOuterJoin(targetDegree) + .where(0) + .equalTo(0) + .with(new JoinVertexWithVertexDegree()) + .setParallelism(parallelism) + .name("Join zero degree vertices"); + } + + return targetDegree; + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java new file mode 100644 index 0000000000000..267610435d867 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegree.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.directed; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.DegreeCount; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinVertexWithVertexDegree; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.MapEdgeToSourceId; +import org.apache.flink.types.LongValue; + +/** + * Annotates vertices of a directed graph with the out-degree. + * + * @param graph label type + * @param vertex value type + * @param edge value type + */ +public class VertexOutDegree +implements GraphAlgorithm>> { + + // Optional configuration + private boolean includeZeroDegreeVertices = true; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * By default only the edge set is processed for the computation of degree. + * When this flag is set an additional join is performed against the vertex + * set in order to output vertices with an out-degree of zero. + * + * @param includeZeroDegreeVertices whether to output vertices with an + * out-degree of zero + * @return this + */ + public VertexOutDegree setIncludeZeroDegreeVertices(boolean includeZeroDegreeVertices) { + this.includeZeroDegreeVertices = includeZeroDegreeVertices; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public VertexOutDegree setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet> run(Graph input) + throws Exception { + // s + DataSet> sourceIds = input + .getEdges() + .map(new MapEdgeToSourceId()) + .setParallelism(parallelism) + .name("Map edge to source ID"); + + // s, deg(s) + DataSet> sourceDegree = sourceIds + .groupBy(0) + .reduce(new DegreeCount()) + .setParallelism(parallelism) + .name("Degree count"); + + if (includeZeroDegreeVertices) { + sourceDegree = input + .getVertices() + .leftOuterJoin(sourceDegree) + .where(0) + .equalTo(0) + .with(new JoinVertexWithVertexDegree()) + .setParallelism(parallelism) + .name("Join zero degree vertices"); + } + + return sourceDegree; + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/package-info.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/package-info.java new file mode 100644 index 0000000000000..f25fd90bb8232 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/package-info.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * The degree annotation package provides a collection of edge-centric graph + * algorithms for counting the vertex degree of directed and undirected graphs. + * + * Undirected graphs have the property that for every vertex the in-degree is + * equivalent to the out-degree. + * + * The undirected graph algorithms are: + * {@code VertexDegree} annotates vertices as + * {@code EdgeSourceDegree} annotates edges as + * {@code EdgeTargetDegree} annotates edges as + * {@code EdgeDegreePair} annotates edges as + * + * The directed graph algorithms are: + * {@code VertexOutDegree} annotates vertices as + * {@code VertexInDegree} annotates vertices as + * {@code VertexDegreePair} annotates vertices as + * + * A directed graph edge has four possible degrees: source out- and in-degree + * and target out- and in-degree. This gives 2^4 - 1 = 15 ways to annotate + * a directed edge. + */ +package org.apache.flink.graph.asm.degree.annotate; diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java new file mode 100644 index 0000000000000..d85b4fab722d7 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePair.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinEdgeDegreeWithVertexDegree; +import org.apache.flink.types.LongValue; + +/** + * Annotates edges of an undirected graph with the degree of both the source and target degree ID. + * + * @param ID type + * @param vertex value type + * @param edge value type + */ +public class EdgeDegreePair +implements GraphAlgorithm>>> { + + // Optional configuration + protected boolean reduceOnTargetId = false; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * The degree can be counted from either the edge source or target IDs. + * By default the source IDs are counted. Reducing on target IDs may + * optimize the algorithm if the input edge list is sorted by target ID. + * + * @param reduceOnTargetId set to {@code true} if the input edge list + * is sorted by target ID + * @return this + */ + public EdgeDegreePair setReduceOnTargetId(boolean reduceOnTargetId) { + this.reduceOnTargetId = reduceOnTargetId; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public EdgeDegreePair setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet>> run(Graph input) + throws Exception { + // s, t, deg(s) + DataSet>> edgeSourceDegrees = input + .run(new EdgeSourceDegree() + .setReduceOnTargetId(reduceOnTargetId) + .setParallelism(parallelism)); + + DataSet> vertexDegrees; + + if (reduceOnTargetId) { + // t, d(t) + vertexDegrees = input + .run(new VertexDegree() + .setReduceOnTargetId(true) + .setParallelism(parallelism)); + } else { + // s, d(s) + vertexDegrees = input + .run(new VertexDegree() + .setParallelism(parallelism)); + } + + // s, t, (deg(s), deg(t)) + return edgeSourceDegrees + .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) + .where(1) + .equalTo(0) + .with(new JoinEdgeDegreeWithVertexDegree()) + .setParallelism(parallelism) + .name("Edge target degree"); + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java new file mode 100644 index 0000000000000..d200dd38c55a2 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegree.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinEdgeWithVertexDegree; +import org.apache.flink.types.LongValue; + +/** + * Annotates edges of an undirected graph with degree of the source ID. + * + * @param ID type + * @param vertex value type + * @param edge value type + */ +public class EdgeSourceDegree +implements GraphAlgorithm>>> { + + // Optional configuration + private boolean reduceOnTargetId = false; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * The degree can be counted from either the edge source or target IDs. + * By default the source IDs are counted. Reducing on target IDs may + * optimize the algorithm if the input edge list is sorted by target ID. + * + * @param reduceOnTargetId set to {@code true} if the input edge list + * is sorted by target ID + * @return this + */ + public EdgeSourceDegree setReduceOnTargetId(boolean reduceOnTargetId) { + this.reduceOnTargetId = reduceOnTargetId; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public EdgeSourceDegree setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet>> run(Graph input) + throws Exception { + DataSet> vertexDegrees; + + if (reduceOnTargetId) { + // t, d(t) + vertexDegrees = input + .run(new VertexDegree() + .setReduceOnTargetId(true) + .setParallelism(parallelism)); + } else { + // s, d(s) + vertexDegrees = input + .run(new VertexDegree() + .setParallelism(parallelism)); + } + + // s, t, d(s) + return input.getEdges() + .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) + .where(0) + .equalTo(0) + .with(new JoinEdgeWithVertexDegree()) + .setParallelism(parallelism) + .name("Edge source degree"); + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java new file mode 100644 index 0000000000000..8716f74d97158 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegree.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinEdgeWithVertexDegree; +import org.apache.flink.types.LongValue; + +/** + * Annotates edges of an undirected graph with degree of the target ID. + * + * @param ID type + * @param vertex value type + * @param edge value type + */ +public class EdgeTargetDegree +implements GraphAlgorithm>>> { + + // Optional configuration + private boolean reduceOnSourceId = false; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * The degree can be counted from either the edge source or target IDs. + * By default the target IDs are counted. Reducing on source IDs may + * optimize the algorithm if the input edge list is sorted by source ID. + * + * @param reduceOnSourceId set to {@code true} if the input edge list + * is sorted by target ID + * @return this + */ + public EdgeTargetDegree setReduceOnSourceId(boolean reduceOnSourceId) { + this.reduceOnSourceId = reduceOnSourceId; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public EdgeTargetDegree setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet>> run(Graph input) + throws Exception { + DataSet> vertexDegrees; + + if (reduceOnSourceId) { + // s, d(s) + vertexDegrees = input + .run(new VertexDegree() + .setParallelism(parallelism)); + } else { + // t, d(t) + vertexDegrees = input + .run(new VertexDegree() + .setReduceOnTargetId(true) + .setParallelism(parallelism)); + } + + // s, t, d(t) + return input.getEdges() + .join(vertexDegrees, JoinHint.REPARTITION_HASH_SECOND) + .where(1) + .equalTo(0) + .with(new JoinEdgeWithVertexDegree()) + .setParallelism(parallelism) + .name("Edge target degree"); + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java new file mode 100644 index 0000000000000..518afaa973182 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegree.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.DegreeCount; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.JoinVertexWithVertexDegree; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.MapEdgeToSourceId; +import org.apache.flink.graph.asm.degree.annotate.DegreeAnnotationFunctions.MapEdgeToTargetId; +import org.apache.flink.types.LongValue; + +/** + * Annotates vertices of an undirected graph with the degree. + * + * @param ID type + * @param vertex value type + * @param edge value type + */ +public class VertexDegree +implements GraphAlgorithm>> { + + // Optional configuration + private boolean includeZeroDegreeVertices = false; + + private boolean reduceOnTargetId = false; + + private int parallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * By default only the edge set is processed for the computation of degree. + * When this flag is set an additional join is performed against the vertex + * set in order to output vertices with a degree of zero. + * + * @param includeZeroDegreeVertices whether to output vertices with a + * degree of zero + * @return this + */ + public VertexDegree setIncludeZeroDegreeVertices(boolean includeZeroDegreeVertices) { + this.includeZeroDegreeVertices = includeZeroDegreeVertices; + + return this; + } + + /** + * The degree can be counted from either the edge source or target IDs. + * By default the source IDs are counted. Reducing on target IDs may + * optimize the algorithm if the input edge list is sorted by target ID. + * + * @param reduceOnTargetId set to {@code true} if the input edge list + * is sorted by target ID + * @return this + */ + public VertexDegree setReduceOnTargetId(boolean reduceOnTargetId) { + this.reduceOnTargetId = reduceOnTargetId; + + return this; + } + + /** + * Override the operator parallelism. + * + * @param parallelism operator parallelism + * @return this + */ + public VertexDegree setParallelism(int parallelism) { + this.parallelism = parallelism; + + return this; + } + + @Override + public DataSet> run(Graph input) + throws Exception { + MapFunction, Vertex> mapEdgeToId = reduceOnTargetId ? + new MapEdgeToTargetId() : new MapEdgeToSourceId(); + + // v + DataSet> vertexIds = input + .getEdges() + .map(mapEdgeToId) + .setParallelism(parallelism) + .name("Map edge to vertex ID"); + + // v, deg(v) + DataSet> degree = vertexIds + .groupBy(0) + .reduce(new DegreeCount()) + .setParallelism(parallelism) + .name("Degree count"); + + if (includeZeroDegreeVertices) { + degree = input + .getVertices() + .leftOuterJoin(degree) + .where(0) + .equalTo(0) + .with(new JoinVertexWithVertexDegree()) + .setParallelism(parallelism) + .name("Join zero degree vertices"); + } + + return degree; + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java new file mode 100644 index 0000000000000..0d98ffce6feab --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/AsmTestBase.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.generator.CompleteGraph; +import org.apache.flink.graph.generator.EmptyGraph; +import org.apache.flink.graph.generator.RMatGraph; +import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Before; + +import java.util.LinkedList; +import java.util.List; + +public class AsmTestBase { + + protected ExecutionEnvironment env; + + // simple graph + protected Graph directedSimpleGraph; + + protected Graph undirectedSimpleGraph; + + // complete graph + protected final long completeGraphVertexCount = 47; + + protected Graph completeGraph; + + // empty graph + protected Graph emptyGraph; + + // RMat graph + protected Graph directedRMatGraph; + + protected Graph undirectedRMatGraph; + + @Before + public void setup() { + env = ExecutionEnvironment.createCollectionsEnvironment(); + + // the "fish" graph + Object[][] edges = new Object[][] { + new Object[]{0, 1}, + new Object[]{0, 2}, + new Object[]{1, 2}, + new Object[]{1, 3}, + new Object[]{2, 3}, + new Object[]{3, 4}, + new Object[]{3, 5}, + }; + + List> directedEdgeList = new LinkedList<>(); + + for (Object[] edge : edges) { + directedEdgeList.add(new Edge<>(new IntValue((int) edge[0]), new IntValue((int) edge[1]), NullValue.getInstance())); + } + + directedSimpleGraph = Graph.fromCollection(directedEdgeList, env); + undirectedSimpleGraph = directedSimpleGraph + .getUndirected(); + + // complete graph + completeGraph = new CompleteGraph(env, completeGraphVertexCount) + .generate(); + + // empty graph + emptyGraph = new EmptyGraph(env, 3) + .generate(); + + // RMat graph + long rmatVertexCount = 1 << 10; + long rmatEdgeCount = 16 * rmatVertexCount; + + directedRMatGraph = new RMatGraph<>(env, new JDKRandomGeneratorFactory(), rmatVertexCount, rmatEdgeCount) + .generate(); + + undirectedRMatGraph = new RMatGraph<>(env, new JDKRandomGeneratorFactory(), rmatVertexCount, rmatEdgeCount) + .setSimpleGraph(true, false) + .generate(); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePairTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePairTest.java new file mode 100644 index 0000000000000..94005326eda42 --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexDegreePairTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.directed; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class VertexDegreePairTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + DataSet>> vertexDegrees = directedSimpleGraph + .run(new VertexDegreePair()); + + String expectedResult = + "(0,(2,0))\n" + + "(1,(2,1))\n" + + "(2,(1,2))\n" + + "(3,(2,2))\n" + + "(4,(0,1))\n" + + "(5,(0,1))"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithEmptyGraph() + throws Exception { + DataSet>> vertexDegrees; + + vertexDegrees = emptyGraph + .run(new VertexDegreePair() + .setIncludeZeroDegreeVertices(false)); + + assertEquals(0, vertexDegrees.collect().size()); + + vertexDegrees = emptyGraph + .run(new VertexDegreePair() + .setIncludeZeroDegreeVertices(true)); + + String expectedResult = + "(0,(0,0))\n" + + "(1,(0,0))\n" + + "(2,(0,0))"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode degreePairChecksum = DataSetUtils.checksumHashCode(directedRMatGraph + .run(new VertexDegreePair())); + + assertEquals(902, degreePairChecksum.getCount()); + assertEquals(0x0000000000fc025aL, degreePairChecksum.getChecksum()); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java new file mode 100644 index 0000000000000..577e675d5a272 --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexInDegreeTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.directed; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class VertexInDegreeTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + DataSet> vertexDegrees = directedSimpleGraph + .run(new VertexInDegree()); + + String expectedResult = + "(0,0)\n" + + "(1,1)\n" + + "(2,2)\n" + + "(3,2)\n" + + "(4,1)\n" + + "(5,1)"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithEmptyGraph() + throws Exception { + DataSet> vertexDegrees; + + vertexDegrees = emptyGraph + .run(new VertexInDegree() + .setIncludeZeroDegreeVertices(false)); + + assertEquals(0, vertexDegrees.collect().size()); + + vertexDegrees = emptyGraph + .run(new VertexInDegree() + .setIncludeZeroDegreeVertices(true)); + + String expectedResult = + "(0,0)\n" + + "(1,0)\n" + + "(2,0)"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode inDegreeChecksum = DataSetUtils.checksumHashCode(directedRMatGraph + .run(new VertexInDegree())); + + assertEquals(902, inDegreeChecksum.getCount()); + assertEquals(0x0000000000e1e99cL, inDegreeChecksum.getChecksum()); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java new file mode 100644 index 0000000000000..b5e9ce8f60fd0 --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/directed/VertexOutDegreeTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.directed; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class VertexOutDegreeTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + DataSet> vertexDegrees = directedSimpleGraph + .run(new VertexOutDegree()); + + String expectedResult = + "(0,2)\n" + + "(1,2)\n" + + "(2,1)\n" + + "(3,2)\n" + + "(4,0)\n" + + "(5,0)"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithEmptyGraph() + throws Exception { + DataSet> vertexDegrees; + + vertexDegrees = emptyGraph + .run(new VertexOutDegree() + .setIncludeZeroDegreeVertices(false)); + + assertEquals(0, vertexDegrees.collect().size()); + + vertexDegrees = emptyGraph + .run(new VertexOutDegree() + .setIncludeZeroDegreeVertices(true)); + + String expectedResult = + "(0,0)\n" + + "(1,0)\n" + + "(2,0)"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode outDegreeChecksum = DataSetUtils.checksumHashCode(directedRMatGraph + .run(new VertexOutDegree())); + + assertEquals(902, outDegreeChecksum.getCount()); + assertEquals(0x0000000000e1e99cL, outDegreeChecksum.getChecksum()); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java new file mode 100644 index 0000000000000..3a8636a0c097d --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeDegreePairTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class EdgeDegreePairTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + String expectedResult = + "(0,1,((null),2,3))\n" + + "(0,2,((null),2,3))\n" + + "(1,0,((null),3,2))\n" + + "(1,2,((null),3,3))\n" + + "(1,3,((null),3,4))\n" + + "(2,0,((null),3,2))\n" + + "(2,1,((null),3,3))\n" + + "(2,3,((null),3,4))\n" + + "(3,1,((null),4,3))\n" + + "(3,2,((null),4,3))\n" + + "(3,4,((null),4,1))\n" + + "(3,5,((null),4,1))\n" + + "(4,3,((null),1,4))\n" + + "(5,3,((null),1,4))"; + + DataSet>> sourceDegree = undirectedSimpleGraph + .run(new EdgeDegreePair()); + + TestBaseUtils.compareResultAsText(sourceDegree.collect(), expectedResult); + + DataSet>> targetDegree = undirectedSimpleGraph + .run(new EdgeDegreePair() + .setReduceOnTargetId(true)); + + TestBaseUtils.compareResultAsText(targetDegree.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode sourceDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new EdgeDegreePair())); + + assertEquals(20884, sourceDegreeChecksum.getCount()); + assertEquals(0x00000001e051efe4L, sourceDegreeChecksum.getChecksum()); + + ChecksumHashCode targetDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new EdgeDegreePair() + .setReduceOnTargetId(true))); + + assertEquals(sourceDegreeChecksum, targetDegreeChecksum); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java new file mode 100644 index 0000000000000..96714615a2046 --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeSourceDegreeTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class EdgeSourceDegreeTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + String expectedResult = + "(0,1,((null),2))\n" + + "(0,2,((null),2))\n" + + "(1,0,((null),3))\n" + + "(1,2,((null),3))\n" + + "(1,3,((null),3))\n" + + "(2,0,((null),3))\n" + + "(2,1,((null),3))\n" + + "(2,3,((null),3))\n" + + "(3,1,((null),4))\n" + + "(3,2,((null),4))\n" + + "(3,4,((null),4))\n" + + "(3,5,((null),4))\n" + + "(4,3,((null),1))\n" + + "(5,3,((null),1))"; + + DataSet>> sourceDegree = undirectedSimpleGraph + .run(new EdgeSourceDegree()); + + TestBaseUtils.compareResultAsText(sourceDegree.collect(), expectedResult); + + DataSet>> targetDegree = undirectedSimpleGraph + .run(new EdgeSourceDegree() + .setReduceOnTargetId(true)); + + TestBaseUtils.compareResultAsText(targetDegree.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode sourceDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new EdgeSourceDegree())); + + assertEquals(20884, sourceDegreeChecksum.getCount()); + assertEquals(0x000000019d8f0070L, sourceDegreeChecksum.getChecksum()); + + ChecksumHashCode targetDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new EdgeSourceDegree() + .setReduceOnTargetId(true))); + + assertEquals(sourceDegreeChecksum, targetDegreeChecksum); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java new file mode 100644 index 0000000000000..54f2063069acc --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/EdgeTargetDegreeTest.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class EdgeTargetDegreeTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + String expectedResult = + "(0,1,((null),3))\n" + + "(0,2,((null),3))\n" + + "(1,0,((null),2))\n" + + "(1,2,((null),3))\n" + + "(1,3,((null),4))\n" + + "(2,0,((null),2))\n" + + "(2,1,((null),3))\n" + + "(2,3,((null),4))\n" + + "(3,1,((null),3))\n" + + "(3,2,((null),3))\n" + + "(3,4,((null),1))\n" + + "(3,5,((null),1))\n" + + "(4,3,((null),4))\n" + + "(5,3,((null),4))"; + + DataSet>> sourceDegree = undirectedSimpleGraph + .run(new EdgeTargetDegree()); + + TestBaseUtils.compareResultAsText(sourceDegree.collect(), expectedResult); + + DataSet>> targetDegree = undirectedSimpleGraph + .run(new EdgeTargetDegree() + .setReduceOnSourceId(true)); + + TestBaseUtils.compareResultAsText(targetDegree.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode sourceDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new EdgeSourceDegree())); + + assertEquals(20884, sourceDegreeChecksum.getCount()); + assertEquals(0x000000019d8f0070L, sourceDegreeChecksum.getChecksum()); + + ChecksumHashCode targetDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new EdgeTargetDegree() + .setReduceOnSourceId(true))); + + assertEquals(sourceDegreeChecksum, targetDegreeChecksum); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java new file mode 100644 index 0000000000000..307ff4c2031d9 --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/degree/annotate/undirected/VertexDegreeTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.degree.annotate.undirected; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class VertexDegreeTest +extends AsmTestBase { + + @Test + public void testWithSimpleGraph() + throws Exception { + String expectedResult = + "(0,2)\n" + + "(1,3)\n" + + "(2,3)\n" + + "(3,4)\n" + + "(4,1)\n" + + "(5,1)"; + + DataSet> sourceDegrees = undirectedSimpleGraph + .run(new VertexDegree()); + + TestBaseUtils.compareResultAsText(sourceDegrees.collect(), expectedResult); + + DataSet> targetDegrees = undirectedSimpleGraph + .run(new VertexDegree() + .setReduceOnTargetId(true)); + + TestBaseUtils.compareResultAsText(targetDegrees.collect(), expectedResult); + } + + @Test + public void testWithCompleteGraph() + throws Exception { + long expectedDegree = completeGraphVertexCount - 1; + + DataSet> sourceDegrees = completeGraph + .run(new VertexDegree()); + + for (Vertex vertex : sourceDegrees.collect()) { + assertEquals(expectedDegree, vertex.getValue().getValue()); + } + + DataSet> targetDegrees = completeGraph + .run(new VertexDegree() + .setReduceOnTargetId(true)); + + for (Vertex vertex : targetDegrees.collect()) { + assertEquals(expectedDegree, vertex.getValue().getValue()); + } + } + + @Test + public void testWithEmptyGraph() + throws Exception { + DataSet> vertexDegrees; + + vertexDegrees = emptyGraph + .run(new VertexDegree() + .setIncludeZeroDegreeVertices(false)); + + assertEquals(0, vertexDegrees.collect().size()); + + vertexDegrees = emptyGraph + .run(new VertexDegree() + .setIncludeZeroDegreeVertices(true)); + + String expectedResult = + "(0,0)\n" + + "(1,0)\n" + + "(2,0)"; + + TestBaseUtils.compareResultAsText(vertexDegrees.collect(), expectedResult); + } + + @Test + public void testWithRMatGraph() + throws Exception { + ChecksumHashCode sourceDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new VertexDegree())); + + assertEquals(902, sourceDegreeChecksum.getCount()); + assertEquals(0x0000000000e1fb30L, sourceDegreeChecksum.getChecksum()); + + ChecksumHashCode targetDegreeChecksum = DataSetUtils.checksumHashCode(undirectedRMatGraph + .run(new VertexDegree() + .setReduceOnTargetId(true))); + + assertEquals(sourceDegreeChecksum, targetDegreeChecksum); + } +} From 35e61a53db03477e54f6656af8b4cba96a8aee2a Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Wed, 4 May 2016 16:51:23 -0400 Subject: [PATCH 06/70] [FLINK-3877] [gelly] Create TranslateFunction interface for Graph translators The TranslateFunction interface is similar to MapFunction but may be called multiple times before serialization. This closes #1968 --- docs/apis/batch/libs/gelly.md | 6 +- .../org/apache/flink/graph/scala/Graph.scala | 34 +++++---- .../java/org/apache/flink/graph/Graph.java | 7 +- .../asm/translate/LongValueAddOffset.java | 15 ++-- .../asm/translate/LongValueToIntValue.java | 15 ++-- .../asm/translate/LongValueToStringValue.java | 15 ++-- .../asm/translate/RichTranslateFunction.java | 39 ++++++++++ .../flink/graph/asm/translate/Translate.java | 74 +++++++++---------- .../asm/translate/TranslateEdgeValues.java | 9 +-- .../asm/translate/TranslateFunction.java | 50 +++++++++++++ .../asm/translate/TranslateGraphIds.java | 9 +-- .../asm/translate/TranslateVertexValues.java | 9 +-- .../graph/asm/translate/TranslateTest.java | 48 ++++++------ 13 files changed, 212 insertions(+), 118 deletions(-) create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/RichTranslateFunction.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateFunction.java diff --git a/docs/apis/batch/libs/gelly.md b/docs/apis/batch/libs/gelly.md index 79aa68edf9a46..c68001bcc9ffd 100644 --- a/docs/apis/batch/libs/gelly.md +++ b/docs/apis/batch/libs/gelly.md @@ -2193,7 +2193,7 @@ DataSet>> pairDegree = graph translate.
TranslateGraphIds -

Translate vertex and edge IDs using the given MapFunction.

+

Translate vertex and edge IDs using the given TranslateFunction.

{% highlight java %} graph.run(new TranslateGraphIds(new LongValueToStringValue())); {% endhighlight %} @@ -2203,7 +2203,7 @@ graph.run(new TranslateGraphIds(new LongValueToStringValue())); translate.
TranslateVertexValues -

Translate vertex values using the given MapFunction.

+

Translate vertex values using the given TranslateFunction.

{% highlight java %} graph.run(new TranslateVertexValues(new LongValueAddOffset(vertexCount))); {% endhighlight %} @@ -2213,7 +2213,7 @@ graph.run(new TranslateVertexValues(new LongValueAddOffset(vertexCount))); translate.
TranslateEdgeValues -

Translate edge values using the given MapFunction.

+

Translate edge values using the given TranslateFunction.

{% highlight java %} graph.run(new TranslateEdgeValues(new Nullify())); {% endhighlight %} diff --git a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala index 5e1431972bbe1..f31619decccbd 100644 --- a/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala +++ b/flink-libraries/flink-gelly-scala/src/main/scala/org/apache/flink/graph/scala/Graph.scala @@ -24,10 +24,12 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.{tuple => jtuple} import org.apache.flink.api.scala._ import org.apache.flink.graph._ +import org.apache.flink.graph.asm.translate.TranslateFunction import org.apache.flink.graph.validation.GraphValidator import org.apache.flink.graph.gsa.{ApplyFunction, GSAConfiguration, GatherFunction, SumFunction} import org.apache.flink.graph.spargel.{MessagingFunction, ScatterGatherConfiguration, VertexUpdateFunction} import org.apache.flink.{graph => jg} + import _root_.scala.collection.JavaConverters._ import _root_.scala.reflect.ClassTag import org.apache.flink.types.NullValue @@ -412,7 +414,7 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { * @param translator implements conversion from K to NEW * @return graph with translated vertex and edge IDs */ - def translateGraphIds[NEW: TypeInformation : ClassTag](translator: MapFunction[K, NEW]): + def translateGraphIds[NEW: TypeInformation : ClassTag](translator: TranslateFunction[K, NEW]): Graph[NEW, VV, EV] = { new Graph[NEW, VV, EV](jgraph.translateGraphIds(translator)) } @@ -423,15 +425,15 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { * @param fun implements conversion from K to NEW * @return graph with translated vertex and edge IDs */ - def translateGraphIds[NEW: TypeInformation : ClassTag](fun: K => NEW): + def translateGraphIds[NEW: TypeInformation : ClassTag](fun: (K, NEW) => NEW): Graph[NEW, VV, EV] = { - val mapper: MapFunction[K, NEW] = new MapFunction[K, NEW] { + val translator: TranslateFunction[K, NEW] = new TranslateFunction[K, NEW] { val cleanFun = clean(fun) - def map(in: K): NEW = cleanFun(in) + def translate(in: K, reuse: NEW): NEW = cleanFun(in, reuse) } - new Graph[NEW, VV, EV](jgraph.translateGraphIds(mapper)) + new Graph[NEW, VV, EV](jgraph.translateGraphIds(translator)) } /** @@ -440,8 +442,8 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { * @param translator implements conversion from VV to NEW * @return graph with translated vertex values */ - def translateVertexValues[NEW: TypeInformation : ClassTag](translator: MapFunction[VV, NEW]): - Graph[K, NEW, EV] = { + def translateVertexValues[NEW: TypeInformation : ClassTag](translator: + TranslateFunction[VV, NEW]): Graph[K, NEW, EV] = { new Graph[K, NEW, EV](jgraph.translateVertexValues(translator)) } @@ -451,15 +453,15 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { * @param fun implements conversion from VV to NEW * @return graph with translated vertex values */ - def translateVertexValues[NEW: TypeInformation : ClassTag](fun: VV => NEW): + def translateVertexValues[NEW: TypeInformation : ClassTag](fun: (VV, NEW) => NEW): Graph[K, NEW, EV] = { - val mapper: MapFunction[VV, NEW] = new MapFunction[VV, NEW] { + val translator: TranslateFunction[VV, NEW] = new TranslateFunction[VV, NEW] { val cleanFun = clean(fun) - def map(in: VV): NEW = cleanFun(in) + def translate(in: VV, reuse: NEW): NEW = cleanFun(in, reuse) } - new Graph[K, NEW, EV](jgraph.translateVertexValues(mapper)) + new Graph[K, NEW, EV](jgraph.translateVertexValues(translator)) } /** @@ -468,7 +470,7 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { * @param translator implements conversion from EV to NEW * @return graph with translated edge values */ - def translateEdgeValues[NEW: TypeInformation : ClassTag](translator: MapFunction[EV, NEW]): + def translateEdgeValues[NEW: TypeInformation : ClassTag](translator: TranslateFunction[EV, NEW]): Graph[K, VV, NEW] = { new Graph[K, VV, NEW](jgraph.translateEdgeValues(translator)) } @@ -479,15 +481,15 @@ TypeInformation : ClassTag](jgraph: jg.Graph[K, VV, EV]) { * @param fun implements conversion from EV to NEW * @return graph with translated edge values */ - def translateEdgeValues[NEW: TypeInformation : ClassTag](fun: EV => NEW): + def translateEdgeValues[NEW: TypeInformation : ClassTag](fun: (EV, NEW) => NEW): Graph[K, VV, NEW] = { - val mapper: MapFunction[EV, NEW] = new MapFunction[EV, NEW] { + val translator: TranslateFunction[EV, NEW] = new TranslateFunction[EV, NEW] { val cleanFun = clean(fun) - def map(in: EV): NEW = cleanFun(in) + def translate(in: EV, reuse: NEW): NEW = cleanFun(in, reuse) } - new Graph[K, VV, NEW](jgraph.translateEdgeValues(mapper)) + new Graph[K, VV, NEW](jgraph.translateEdgeValues(translator)) } /** diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java index fbc0c2374215f..b17f7a587c701 100755 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/Graph.java @@ -47,6 +47,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.graph.asm.translate.TranslateEdgeValues; +import org.apache.flink.graph.asm.translate.TranslateFunction; import org.apache.flink.graph.asm.translate.TranslateGraphIds; import org.apache.flink.graph.asm.translate.TranslateVertexValues; import org.apache.flink.graph.gsa.ApplyFunction; @@ -557,7 +558,7 @@ public Graph mapEdges(final MapFunction, NV> mapper) * @return graph with translated vertex and edge IDs * @throws Exception */ - public Graph translateGraphIds(MapFunction translator) throws Exception { + public Graph translateGraphIds(TranslateFunction translator) throws Exception { return run(new TranslateGraphIds(translator)); } @@ -569,7 +570,7 @@ public Graph translateGraphIds(MapFunction translator * @return graph with translated vertex values * @throws Exception */ - public Graph translateVertexValues(MapFunction translator) throws Exception { + public Graph translateVertexValues(TranslateFunction translator) throws Exception { return run(new TranslateVertexValues(translator)); } @@ -581,7 +582,7 @@ public Graph translateVertexValues(MapFunction transl * @return graph with translated edge values * @throws Exception */ - public Graph translateEdgeValues(MapFunction translator) throws Exception { + public Graph translateEdgeValues(TranslateFunction translator) throws Exception { return run(new TranslateEdgeValues(translator)); } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueAddOffset.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueAddOffset.java index b21fe8415992a..4b553079a5cc7 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueAddOffset.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueAddOffset.java @@ -18,19 +18,16 @@ package org.apache.flink.graph.asm.translate; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.types.LongValue; /** * Translate {@link LongValue} by adding a constant offset value. */ public class LongValueAddOffset -implements MapFunction { +implements TranslateFunction { private final long offset; - private LongValue output = new LongValue(); - /** * Translate {@link LongValue} by adding a constant offset value. * @@ -41,9 +38,13 @@ public LongValueAddOffset(long offset) { } @Override - public LongValue map(LongValue value) + public LongValue translate(LongValue value, LongValue reuse) throws Exception { - output.setValue(offset + value.getValue()); - return output; + if (reuse == null) { + reuse = new LongValue(); + } + + reuse.setValue(offset + value.getValue()); + return reuse; } } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToIntValue.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToIntValue.java index c14fe0472ff46..7ed63b7399f47 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToIntValue.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToIntValue.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.asm.translate; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.types.IntValue; import org.apache.flink.types.LongValue; @@ -28,12 +27,10 @@ * Throws {@link RuntimeException} for integer overflow. */ public class LongValueToIntValue -implements MapFunction { - - private IntValue output = new IntValue(); +implements TranslateFunction { @Override - public IntValue map(LongValue value) + public IntValue translate(LongValue value, IntValue reuse) throws Exception { long val = value.getValue(); @@ -41,7 +38,11 @@ public IntValue map(LongValue value) throw new RuntimeException("LongValue input overflows IntValue output"); } - output.setValue((int) val); - return output; + if (reuse == null) { + reuse = new IntValue(); + } + + reuse.setValue((int) val); + return reuse; } } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToStringValue.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToStringValue.java index 3d9a5cbb9ce28..bdcf37c97c112 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToStringValue.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/LongValueToStringValue.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.asm.translate; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.types.LongValue; import org.apache.flink.types.StringValue; @@ -26,14 +25,16 @@ * Translate {@link LongValue} to {@link StringValue}. */ public class LongValueToStringValue -implements MapFunction { - - private StringValue output = new StringValue(); +implements TranslateFunction { @Override - public StringValue map(LongValue value) + public StringValue translate(LongValue value, StringValue reuse) throws Exception { - output.setValue(Long.toString(value.getValue())); - return output; + if (reuse == null) { + reuse = new StringValue(); + } + + reuse.setValue(Long.toString(value.getValue())); + return reuse; } } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/RichTranslateFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/RichTranslateFunction.java new file mode 100644 index 0000000000000..a89b42a8d36b1 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/RichTranslateFunction.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.translate; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.RichFunction; + +/** + * Rich variant of the {@link TranslateFunction}. As a {@link RichFunction}, it gives access to the + * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods: + * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and + * {@link RichFunction#close()}. + * + * @param Type of the input elements. + * @param Type of the returned elements. + */ +public abstract class RichTranslateFunction extends AbstractRichFunction implements TranslateFunction { + + private static final long serialVersionUID = 1L; + + @Override + public abstract OUT translate(IN value, OUT reuse) throws Exception; +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java index 585472da1cd75..d87c949a5029e 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/Translate.java @@ -43,7 +43,7 @@ public class Translate { // -------------------------------------------------------------------------------------------- /** - * Translate {@link Vertex} IDs using the given {@link MapFunction}. + * Translate {@link Vertex} IDs using the given {@link TranslateFunction}. * * @param vertices input vertices * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -52,12 +52,12 @@ public class Translate { * @param vertex value type * @return translated vertices */ - public static DataSet> translateVertexIds(DataSet> vertices, MapFunction translator) { + public static DataSet> translateVertexIds(DataSet> vertices, TranslateFunction translator) { return translateVertexIds(vertices, translator, PARALLELISM_UNKNOWN); } /** - * Translate {@link Vertex} IDs using the given {@link MapFunction}. + * Translate {@link Vertex} IDs using the given {@link TranslateFunction}. * * @param vertices input vertices * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -68,14 +68,14 @@ public static DataSet> translateVertexIds(DataSet * @return translated vertices */ @SuppressWarnings("unchecked") - public static DataSet> translateVertexIds(DataSet> vertices, MapFunction translator, int parallelism) { + public static DataSet> translateVertexIds(DataSet> vertices, TranslateFunction translator, int parallelism) { Preconditions.checkNotNull(vertices); Preconditions.checkNotNull(translator); Preconditions.checkArgument(parallelism > 0 || parallelism == PARALLELISM_DEFAULT || parallelism == PARALLELISM_UNKNOWN, "The parallelism must be greater than zero."); Class> vertexClass = (Class>)(Class) Vertex.class; - TypeInformation newType = TypeExtractor.createTypeInfo(MapFunction.class, translator.getClass(), 1, null, null); + TypeInformation newType = TypeExtractor.createTypeInfo(TranslateFunction.class, translator.getClass(), 1, null, null); TypeInformation vertexValueType = ((TupleTypeInfo>) vertices.getType()).getTypeAt(1); TupleTypeInfo> returnType = new TupleTypeInfo<>(vertexClass, newType, vertexValueType); @@ -88,7 +88,7 @@ public static DataSet> translateVertexIds(DataSet } /** - * Translate {@link Vertex} IDs using the given {@link MapFunction}. + * Translate {@link Vertex} IDs using the given {@link TranslateFunction}. * * @param old vertex ID type * @param new vertex ID type @@ -96,18 +96,18 @@ public static DataSet> translateVertexIds(DataSet */ @ForwardedFields("1") private static class TranslateVertexId - extends WrappingFunction> + extends WrappingFunction> implements MapFunction, Vertex> { private Vertex vertex = new Vertex<>(); - public TranslateVertexId(MapFunction translator) { + public TranslateVertexId(TranslateFunction translator) { super(translator); } @Override public Vertex map(Vertex value) throws Exception { - vertex.f0 = wrappedFunction.map(value.f0); + vertex.f0 = wrappedFunction.translate(value.f0, vertex.f0); vertex.f1 = value.f1; return vertex; @@ -119,7 +119,7 @@ public Vertex map(Vertex value) // -------------------------------------------------------------------------------------------- /** - * Translate {@link Edge} IDs using the given {@link MapFunction}. + * Translate {@link Edge} IDs using the given {@link TranslateFunction}. * * @param edges input edges * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -128,12 +128,12 @@ public Vertex map(Vertex value) * @param edge value type * @return translated edges */ - public static DataSet> translateEdgeIds(DataSet> edges, MapFunction translator) { + public static DataSet> translateEdgeIds(DataSet> edges, TranslateFunction translator) { return translateEdgeIds(edges, translator, PARALLELISM_UNKNOWN); } /** - * Translate {@link Edge} IDs using the given {@link MapFunction}. + * Translate {@link Edge} IDs using the given {@link TranslateFunction}. * * @param edges input edges * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -144,14 +144,14 @@ public static DataSet> translateEdgeIds(DataSet DataSet> translateEdgeIds(DataSet> edges, MapFunction translator, int parallelism) { + public static DataSet> translateEdgeIds(DataSet> edges, TranslateFunction translator, int parallelism) { Preconditions.checkNotNull(edges); Preconditions.checkNotNull(translator); Preconditions.checkArgument(parallelism > 0 || parallelism == PARALLELISM_DEFAULT || parallelism == PARALLELISM_UNKNOWN, "The parallelism must be greater than zero."); Class> edgeClass = (Class>)(Class) Edge.class; - TypeInformation newType = TypeExtractor.createTypeInfo(MapFunction.class, translator.getClass(), 1, null, null); + TypeInformation newType = TypeExtractor.createTypeInfo(TranslateFunction.class, translator.getClass(), 1, null, null); TypeInformation edgeValueType = ((TupleTypeInfo>) edges.getType()).getTypeAt(2); TupleTypeInfo> returnType = new TupleTypeInfo<>(edgeClass, newType, newType, edgeValueType); @@ -164,7 +164,7 @@ public static DataSet> translateEdgeIds(DataSet old edge ID type * @param new edge ID type @@ -172,19 +172,19 @@ public static DataSet> translateEdgeIds(DataSet - extends WrappingFunction> + extends WrappingFunction> implements MapFunction, Edge> { private Edge edge = new Edge<>(); - public TranslateEdgeId(MapFunction translator) { + public TranslateEdgeId(TranslateFunction translator) { super(translator); } @Override public Edge map(Edge value) throws Exception { - edge.f0 = wrappedFunction.map(value.f0); - edge.f1 = wrappedFunction.map(value.f1); + edge.f0 = wrappedFunction.translate(value.f0, edge.f0); + edge.f1 = wrappedFunction.translate(value.f1, edge.f1); edge.f2 = value.f2; return edge; @@ -196,7 +196,7 @@ public Edge map(Edge value) // -------------------------------------------------------------------------------------------- /** - * Translate {@link Vertex} values using the given {@link MapFunction}. + * Translate {@link Vertex} values using the given {@link TranslateFunction}. * * @param vertices input vertices * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -205,12 +205,12 @@ public Edge map(Edge value) * @param new vertex value type * @return translated vertices */ - public static DataSet> translateVertexValues(DataSet> vertices, MapFunction translator) { + public static DataSet> translateVertexValues(DataSet> vertices, TranslateFunction translator) { return translateVertexValues(vertices, translator, PARALLELISM_UNKNOWN); } /** - * Translate {@link Vertex} values using the given {@link MapFunction}. + * Translate {@link Vertex} values using the given {@link TranslateFunction}. * * @param vertices input vertices * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -221,7 +221,7 @@ public static DataSet> translateVertexValues(DataSe * @return translated vertices */ @SuppressWarnings("unchecked") - public static DataSet> translateVertexValues(DataSet> vertices, MapFunction translator, int parallelism) { + public static DataSet> translateVertexValues(DataSet> vertices, TranslateFunction translator, int parallelism) { Preconditions.checkNotNull(vertices); Preconditions.checkNotNull(translator); Preconditions.checkArgument(parallelism > 0 || parallelism == PARALLELISM_DEFAULT || parallelism == PARALLELISM_UNKNOWN, @@ -229,7 +229,7 @@ public static DataSet> translateVertexValues(DataSe Class> vertexClass = (Class>)(Class) Vertex.class; TypeInformation idType = ((TupleTypeInfo>) vertices.getType()).getTypeAt(0); - TypeInformation newType = TypeExtractor.createTypeInfo(MapFunction.class, translator.getClass(), 1, null, null); + TypeInformation newType = TypeExtractor.createTypeInfo(TranslateFunction.class, translator.getClass(), 1, null, null); TupleTypeInfo> returnType = new TupleTypeInfo<>(vertexClass, idType, newType); @@ -241,7 +241,7 @@ public static DataSet> translateVertexValues(DataSe } /** - * Translate {@link Vertex} values using the given {@link MapFunction}. + * Translate {@link Vertex} values using the given {@link TranslateFunction}. * * @param vertex ID type * @param old vertex value type @@ -249,11 +249,11 @@ public static DataSet> translateVertexValues(DataSe */ @ForwardedFields("0") private static class TranslateVertexValue - extends WrappingFunction> + extends WrappingFunction> implements MapFunction, Vertex> { private Vertex vertex = new Vertex<>(); - public TranslateVertexValue(MapFunction translator) { + public TranslateVertexValue(TranslateFunction translator) { super(translator); } @@ -261,7 +261,7 @@ public TranslateVertexValue(MapFunction translator) { public Vertex map(Vertex value) throws Exception { vertex.f0 = value.f0; - vertex.f1 = wrappedFunction.map(value.f1); + vertex.f1 = wrappedFunction.translate(value.f1, vertex.f1); return vertex; } @@ -272,7 +272,7 @@ public Vertex map(Vertex value) // -------------------------------------------------------------------------------------------- /** - * Translate {@link Edge} values using the given {@link MapFunction}. + * Translate {@link Edge} values using the given {@link TranslateFunction}. * * @param edges input edges * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -281,12 +281,12 @@ public Vertex map(Vertex value) * @param new edge value type * @return translated edges */ - public static DataSet> translateEdgeValues(DataSet> edges, MapFunction translator) { + public static DataSet> translateEdgeValues(DataSet> edges, TranslateFunction translator) { return translateEdgeValues(edges, translator, PARALLELISM_UNKNOWN); } /** - * Translate {@link Edge} values using the given {@link MapFunction}. + * Translate {@link Edge} values using the given {@link TranslateFunction}. * * @param edges input edges * @param translator implements conversion from {@code OLD} to {@code NEW} @@ -297,7 +297,7 @@ public static DataSet> translateEdgeValues(DataSet DataSet> translateEdgeValues(DataSet> edges, MapFunction translator, int parallelism) { + public static DataSet> translateEdgeValues(DataSet> edges, TranslateFunction translator, int parallelism) { Preconditions.checkNotNull(edges); Preconditions.checkNotNull(translator); Preconditions.checkArgument(parallelism > 0 || parallelism == PARALLELISM_DEFAULT || parallelism == PARALLELISM_UNKNOWN, @@ -305,7 +305,7 @@ public static DataSet> translateEdgeValues(DataSet> edgeClass = (Class>)(Class) Edge.class; TypeInformation idType = ((TupleTypeInfo>) edges.getType()).getTypeAt(0); - TypeInformation newType = TypeExtractor.createTypeInfo(MapFunction.class, translator.getClass(), 1, null, null); + TypeInformation newType = TypeExtractor.createTypeInfo(TranslateFunction.class, translator.getClass(), 1, null, null); TupleTypeInfo> returnType = new TupleTypeInfo<>(edgeClass, idType, idType, newType); @@ -317,7 +317,7 @@ public static DataSet> translateEdgeValues(DataSet edge ID type * @param old edge value type @@ -325,11 +325,11 @@ public static DataSet> translateEdgeValues(DataSet - extends WrappingFunction> + extends WrappingFunction> implements MapFunction, Edge> { private Edge edge = new Edge<>(); - public TranslateEdgeValue(MapFunction translator) { + public TranslateEdgeValue(TranslateFunction translator) { super(translator); } @@ -338,7 +338,7 @@ public Edge map(Edge value) throws Exception { edge.f0 = value.f0; edge.f1 = value.f1; - edge.f2 = wrappedFunction.map(value.f2); + edge.f2 = wrappedFunction.translate(value.f2, edge.f2); return edge; } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateEdgeValues.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateEdgeValues.java index 3dd04785d3457..1023626c04ecd 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateEdgeValues.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateEdgeValues.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.asm.translate; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.graph.Edge; import org.apache.flink.graph.Graph; @@ -30,7 +29,7 @@ import static org.apache.flink.graph.asm.translate.Translate.translateEdgeValues; /** - * Translate {@link Edge} values using the given {@link MapFunction}. + * Translate {@link Edge} values using the given {@link TranslateFunction}. * * @param vertex ID type * @param vertex value type @@ -41,17 +40,17 @@ public class TranslateEdgeValues implements GraphAlgorithm> { // Required configuration - private MapFunction translator; + private TranslateFunction translator; // Optional configuration private int parallelism = PARALLELISM_UNKNOWN; /** - * Translate {@link Edge} values using the given {@link MapFunction}. + * Translate {@link Edge} values using the given {@link TranslateFunction}. * * @param translator implements conversion from {@code OLD} to {@code NEW} */ - public TranslateEdgeValues(MapFunction translator) { + public TranslateEdgeValues(TranslateFunction translator) { Preconditions.checkNotNull(translator); this.translator = translator; diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateFunction.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateFunction.java new file mode 100644 index 0000000000000..7f495bf8ff581 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateFunction.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.asm.translate; + +import org.apache.flink.api.common.functions.Function; + +import java.io.Serializable; + +/** + * Base interface for Translate functions. Translate functions take elements and transform them, + * element wise. A Translate function always produces a single result element for each input element. + * Typical applications are transcribing between data types or manipulating element values. + *

+ * Translate functions are used within the Graph API and by translating GraphAlgorithms. + * + * @param Type of the input elements. + * @param Type of the returned elements. + */ +public interface TranslateFunction extends Function, Serializable { + + /** + * The translating method. Takes an element from the input data set and transforms + * it into exactly one element. + * + * @param value input value. + * @param reuse value which may be reused for output; if reuse is null then a new output object + * must be instantiated and returned + * @return the transformed value + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. + */ + O translate(T value, O reuse) throws Exception; +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateGraphIds.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateGraphIds.java index 3e12880e83b87..5b0f67fd44c41 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateGraphIds.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateGraphIds.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.asm.translate; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.graph.Edge; import org.apache.flink.graph.Graph; @@ -32,7 +31,7 @@ import static org.apache.flink.graph.asm.translate.Translate.translateVertexIds; /** - * Translate {@link Vertex} and {@link Edge} IDs of a {@link Graph} using the given {@link MapFunction} + * Translate {@link Vertex} and {@link Edge} IDs of a {@link Graph} using the given {@link TranslateFunction} * * @param old graph ID type * @param new graph ID type @@ -43,17 +42,17 @@ public class TranslateGraphIds implements GraphAlgorithm> { // Required configuration - private MapFunction translator; + private TranslateFunction translator; // Optional configuration private int parallelism = PARALLELISM_UNKNOWN; /** - * Translate {@link Vertex} and {@link Edge} IDs of a {@link Graph} using the given {@link MapFunction} + * Translate {@link Vertex} and {@link Edge} IDs of a {@link Graph} using the given {@link TranslateFunction} * * @param translator implements conversion from {@code OLD} to {@code NEW} */ - public TranslateGraphIds(MapFunction translator) { + public TranslateGraphIds(TranslateFunction translator) { Preconditions.checkNotNull(translator); this.translator = translator; diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateVertexValues.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateVertexValues.java index 7556e02b6f216..4572bfe0033ee 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateVertexValues.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/asm/translate/TranslateVertexValues.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.asm.translate; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.graph.Graph; import org.apache.flink.graph.GraphAlgorithm; @@ -30,7 +29,7 @@ import static org.apache.flink.graph.asm.translate.Translate.translateVertexValues; /** - * Translate {@link Vertex} values using the given {@link MapFunction}. + * Translate {@link Vertex} values using the given {@link TranslateFunction}. * * @param vertex ID type * @param old vertex value type @@ -41,17 +40,17 @@ public class TranslateVertexValues implements GraphAlgorithm> { // Required configuration - private MapFunction translator; + private TranslateFunction translator; // Optional configuration private int parallelism = PARALLELISM_UNKNOWN; /** - * Translate {@link Vertex} values using the given {@link MapFunction}. + * Translate {@link Vertex} values using the given {@link TranslateFunction}. * * @param translator implements conversion from {@code OLD} to {@code NEW} */ - public TranslateVertexValues(MapFunction translator) { + public TranslateVertexValues(TranslateFunction translator) { Preconditions.checkNotNull(translator); this.translator = translator; diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/translate/TranslateTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/translate/TranslateTest.java index 2d1ab5281d4da..e13aa6a5a8d0e 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/translate/TranslateTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/asm/translate/TranslateTest.java @@ -40,28 +40,28 @@ public class TranslateTest { private Graph graph; private String expectedVertexResult = - "(0,0)\n" + - "(1,1)\n" + - "(2,2)\n" + - "(3,3)\n" + - "(4,4)\n" + - "(5,5)\n" + - "(6,6)\n" + - "(7,7)\n" + - "(8,8)\n" + - "(9,9)"; + "(0,1)\n" + + "(1,2)\n" + + "(2,3)\n" + + "(3,4)\n" + + "(4,5)\n" + + "(5,6)\n" + + "(6,7)\n" + + "(7,8)\n" + + "(8,9)\n" + + "(9,10)"; private String expectedEdgeResult = - "(0,0,0)\n" + - "(1,1,1)\n" + - "(2,2,2)\n" + - "(3,3,3)\n" + - "(4,4,4)\n" + - "(5,5,5)\n" + - "(6,6,6)\n" + - "(7,7,7)\n" + - "(8,8,8)\n" + - "(9,9,9)"; + "(0,1,2)\n" + + "(1,2,3)\n" + + "(2,3,4)\n" + + "(3,4,5)\n" + + "(4,5,6)\n" + + "(5,6,7)\n" + + "(6,7,8)\n" + + "(7,8,9)\n" + + "(8,9,10)\n" + + "(9,10,11)"; @Before public void setup() { @@ -73,9 +73,11 @@ public void setup() { List> edgeList = new LinkedList<>(); for (long l = 0 ; l < count ; l++) { - LongValue lv = new LongValue(l); - vertexList.add(new Vertex<>(lv, lv)); - edgeList.add(new Edge<>(lv, lv, lv)); + LongValue lv0 = new LongValue(l); + LongValue lv1 = new LongValue(l+1); + LongValue lv2 = new LongValue(l+2); + vertexList.add(new Vertex<>(lv0, lv1)); + edgeList.add(new Edge<>(lv0, lv1, lv2)); } graph = Graph.fromCollection(vertexList, edgeList, env); From 76a162834ee675c7a5d315f1d0264317d4351924 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 6 May 2016 10:40:51 +0200 Subject: [PATCH 07/70] [FLINK-3880] remove mutex for user accumulators hash map This cloes #1976 --- .../flink/runtime/accumulators/AccumulatorRegistry.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java index 0ef3650292b48..97deeb73a0e15 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorRegistry.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -46,8 +45,7 @@ public class AccumulatorRegistry { new HashMap>(); /* User-defined Accumulator values stored for the executing task. */ - private final Map> userAccumulators = - Collections.synchronizedMap(new HashMap>()); + private final Map> userAccumulators = new HashMap<>(); /* The reporter reference that is handed to the reporting tasks. */ private final ReadWriteReporter reporter; From 2444cd6c6f4340e28d84726306471a47c927eb1b Mon Sep 17 00:00:00 2001 From: Tony Baines Date: Sat, 7 May 2016 14:57:15 +0100 Subject: [PATCH 08/70] [FLINK-3155] Update Flink Dockerfile * ubuntu trusty->xenial * jdk 7u51 -> 8u91 * flink 0.10.1 -> 1.0.2 This closes #1969 --- flink-contrib/docker-flink/base/Dockerfile | 6 +++--- flink-contrib/docker-flink/flink/Dockerfile | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-contrib/docker-flink/base/Dockerfile b/flink-contrib/docker-flink/base/Dockerfile index 67a295322f001..2d0704dac5722 100644 --- a/flink-contrib/docker-flink/base/Dockerfile +++ b/flink-contrib/docker-flink/base/Dockerfile @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -FROM ubuntu:trusty +FROM ubuntu:xenial #requirements RUN apt-get update; apt-get install -y curl wget supervisor openssh-server openssh-client nano @@ -24,9 +24,9 @@ RUN apt-get update; apt-get install -y curl wget supervisor openssh-server opens #priviledge separation directory RUN mkdir /var/run/sshd -#install Java 7 Oracle JDK +#install Java 8 Oracle JDK RUN mkdir -p /usr/java/default && \ - curl -Ls 'http://download.oracle.com/otn-pub/java/jdk/7u51-b13/jdk-7u51-linux-x64.tar.gz' -H 'Cookie: oraclelicense=accept-securebackup-cookie' | \ + curl -Ls 'http://download.oracle.com/otn-pub/java/jdk/8u91-b14/jdk-8u91-linux-x64.tar.gz' -H 'Cookie: oraclelicense=accept-securebackup-cookie' | \ tar --strip-components=1 -xz -C /usr/java/default/ ENV JAVA_HOME /usr/java/default/ diff --git a/flink-contrib/docker-flink/flink/Dockerfile b/flink-contrib/docker-flink/flink/Dockerfile index b12ab665bc653..0e39027187169 100644 --- a/flink-contrib/docker-flink/flink/Dockerfile +++ b/flink-contrib/docker-flink/flink/Dockerfile @@ -22,11 +22,11 @@ FROM base RUN ssh-keygen -f ~/.ssh/id_rsa -t rsa -N '' RUN cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys && chmod 600 ~/.ssh/* -##Flink 0.10.0 Installation +##Flink 1.0.2 Installation ###Download: RUN mkdir ~/downloads && cd ~/downloads && \ - wget -q -O - http://mirror.switch.ch/mirror/apache/dist/flink/flink-0.10.1/flink-0.10.1-bin-hadoop27-scala_2.11.tgz| tar -zxvf - -C /usr/local/ -RUN cd /usr/local && ln -s ./flink-0.10.1 flink + wget -q -O - http://mirror.switch.ch/mirror/apache/dist/flink/flink-1.0.2/flink-1.0.2-bin-hadoop27-scala_2.11.tgz| tar -zxvf - -C /usr/local/ +RUN cd /usr/local && ln -s ./flink-1.0.2 flink ENV FLINK_HOME /usr/local/flink ENV PATH $PATH:$FLINK_HOME/bin From 1179e3bc3a3e1b84fa60232376d242831a01b29b Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 27 Apr 2016 14:43:17 +0200 Subject: [PATCH 09/70] [FLINK-3821] [dataSet] Remove Guava usage in flink-java non-test files. - replaced CharSets with StandardCharsets - added checkElementIndex to Flink Preconditions - replaced Guava Preconditions with Flink Preconditions - removed single usages Ints.max() and Joiner() This closes #1938 --- .../org/apache/flink/util/Preconditions.java | 18 +++++++++++++++++- flink-java/pom.xml | 13 +++++++------ .../org/apache/flink/api/java/DataSet.java | 2 +- .../flink/api/java/ExecutionEnvironment.java | 16 +++++++--------- .../mapreduce/HadoopInputFormatBase.java | 11 +++++------ .../flink/api/java/io/CsvInputFormat.java | 7 ++++--- .../apache/flink/api/java/io/CsvReader.java | 3 +-- .../flink/api/java/io/PojoCsvInputFormat.java | 2 +- .../api/java/io/TextValueInputFormat.java | 4 ++-- .../api/java/operators/AggregateOperator.java | 3 +-- .../api/java/operators/CrossOperator.java | 3 +-- .../api/java/operators/DeltaIteration.java | 3 +-- .../flink/api/java/operators/JoinOperator.java | 3 +-- .../api/java/operators/ProjectOperator.java | 3 +-- .../api/java/operators/SortedGrouping.java | 3 +-- .../api/java/operators/UnsortedGrouping.java | 3 +-- .../CombineToGroupCombineWrapper.java | 2 +- .../RichCombineToGroupCombineWrapper.java | 2 +- .../api/java/sampling/BernoulliSampler.java | 2 +- .../api/java/sampling/PoissonSampler.java | 2 +- .../ReservoirSamplerWithReplacement.java | 2 +- .../ReservoirSamplerWithoutReplacement.java | 2 +- .../flink/api/java/utils/DataSetUtils.java | 7 +++++-- .../flink/api/java/utils/ParameterTool.java | 2 +- .../flink/api/java/io/CsvInputFormatTest.java | 5 ++--- .../api/java/sampling/RandomSamplerTest.java | 3 +-- .../flink/api/java/tuple/TupleGenerator.java | 8 ++++---- 27 files changed, 71 insertions(+), 63 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java index a9bd166eb1dc1..25fefe54b53db 100644 --- a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java +++ b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java @@ -218,6 +218,22 @@ public static void checkState(boolean condition, } } + /** + * Ensures that the given index is valid for an array, list or string of the given size. + * + * @param index index to check + * @param size size of the array, list or string + * + * @throws IllegalArgumentException Thrown, if size is negative. + * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to size + */ + public static void checkElementIndex(int index, int size) { + checkArgument(size >= 0, "Size was negative."); + if (index < 0 || index >= size) { + throw new IndexOutOfBoundsException("Index: " + index + ", Size: " + size); + } + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -265,4 +281,4 @@ private static String format(@Nullable String template, @Nullable Object... args /** Private constructor to prevent instantiation */ private Preconditions() {} -} \ No newline at end of file +} diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 5150c5eed5c61..a3689e7433b54 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -53,17 +53,18 @@ under the License. ${asm.version} - - com.google.guava - guava - ${guava.version} - - org.apache.commons commons-math3 + + + com.google.guava + guava + ${guava.version} + test + org.apache.flink diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index b186c3c776926..530de4b9ac7fd 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -18,7 +18,6 @@ package org.apache.flink.api.java; -import com.google.common.base.Preconditions; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.InvalidProgramException; @@ -87,6 +86,7 @@ import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.core.fs.Path; import org.apache.flink.util.AbstractID; +import org.apache.flink.util.Preconditions; import java.io.IOException; import java.util.ArrayList; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index d108d3e00ffc3..be841ac06dfd4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -20,9 +20,6 @@ import com.esotericsoftware.kryo.Serializer; -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; - import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; @@ -60,6 +57,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.types.StringValue; import org.apache.flink.util.NumberSequenceIterator; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.SplittableIterator; import org.apache.flink.util.Visitor; @@ -1090,12 +1088,12 @@ public void postVisit(org.apache.flink.api.common.operators.Operator visitabl } if(LOG.isDebugEnabled()) { - LOG.debug("Registered Kryo types: {}", Joiner.on(',').join(config.getRegisteredKryoTypes())); - LOG.debug("Registered Kryo with Serializers types: {}", Joiner.on(',').join(config.getRegisteredTypesWithKryoSerializers().entrySet())); - LOG.debug("Registered Kryo with Serializer Classes types: {}", Joiner.on(',').join(config.getRegisteredTypesWithKryoSerializerClasses().entrySet())); - LOG.debug("Registered Kryo default Serializers: {}", Joiner.on(',').join(config.getDefaultKryoSerializers().entrySet())); - LOG.debug("Registered Kryo default Serializers Classes {}", Joiner.on(',').join(config.getDefaultKryoSerializerClasses().entrySet())); - LOG.debug("Registered POJO types: {}", Joiner.on(',').join(config.getRegisteredPojoTypes())); + LOG.debug("Registered Kryo types: {}", config.getRegisteredKryoTypes().toString()); + LOG.debug("Registered Kryo with Serializers types: {}", config.getRegisteredTypesWithKryoSerializers().entrySet().toString()); + LOG.debug("Registered Kryo with Serializer Classes types: {}", config.getRegisteredTypesWithKryoSerializerClasses().entrySet().toString()); + LOG.debug("Registered Kryo default Serializers: {}", config.getDefaultKryoSerializers().entrySet().toString()); + LOG.debug("Registered Kryo default Serializers Classes {}", config.getDefaultKryoSerializerClasses().entrySet().toString()); + LOG.debug("Registered POJO types: {}", config.getRegisteredPojoTypes().toString()); // print information about static code analysis LOG.debug("Static code analysis mode: {}", config.getCodeAnalysisMode()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java index 73b11ebd6ca91..0335c23d22ed4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/hadoop/mapreduce/HadoopInputFormatBase.java @@ -30,6 +30,7 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; @@ -49,8 +50,6 @@ import java.util.ArrayList; import java.util.List; -import static com.google.common.base.Preconditions.checkNotNull; - /** * Base class shared between the Java and Scala API of Flink */ @@ -81,10 +80,10 @@ public abstract class HadoopInputFormatBase extends HadoopInputFormatCo protected boolean hasNext; public HadoopInputFormatBase(org.apache.hadoop.mapreduce.InputFormat mapreduceInputFormat, Class key, Class value, Job job) { - super(checkNotNull(job, "Job can not be null").getCredentials()); - this.mapreduceInputFormat = checkNotNull(mapreduceInputFormat); - this.keyClass = checkNotNull(key); - this.valueClass = checkNotNull(value); + super(Preconditions.checkNotNull(job, "Job can not be null").getCredentials()); + this.mapreduceInputFormat = Preconditions.checkNotNull(mapreduceInputFormat); + this.keyClass = Preconditions.checkNotNull(key); + this.valueClass = Preconditions.checkNotNull(value); this.configuration = job.getConfiguration(); HadoopUtils.mergeHadoopConf(configuration); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvInputFormat.java index 4105305d05292..2213d9f8ff6cb 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvInputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvInputFormat.java @@ -18,12 +18,11 @@ package org.apache.flink.api.java.io; -import com.google.common.base.Preconditions; -import com.google.common.primitives.Ints; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.io.GenericCsvInputFormat; import org.apache.flink.core.fs.FileInputSplit; import org.apache.flink.types.parser.FieldParser; +import org.apache.flink.util.Preconditions; import java.io.IOException; import org.apache.flink.core.fs.Path; @@ -133,13 +132,15 @@ protected static boolean[] createDefaultMask(int size) { protected static boolean[] toBooleanMask(int[] sourceFieldIndices) { Preconditions.checkNotNull(sourceFieldIndices); + int max = 0; for (int i : sourceFieldIndices) { if (i < 0) { throw new IllegalArgumentException("Field indices must not be smaller than zero."); } + max = Math.max(i, max); } - boolean[] includedMask = new boolean[Ints.max(sourceFieldIndices) + 1]; + boolean[] includedMask = new boolean[max + 1]; // check if we support parsers for these types for (int i = 0; i < sourceFieldIndices.length; i++) { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java index 3d656a4f7b253..8be5dc23840c7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java @@ -30,13 +30,12 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Preconditions; //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.java.tuple.*; //CHECKSTYLE.ON: AvoidStarImport -import com.google.common.base.Preconditions; - /** * A builder class to instantiate a CSV parsing data source. The CSV reader configures the field types, * the delimiters (row and field), the fields that should be included or skipped, and other flags diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/PojoCsvInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/PojoCsvInputFormat.java index 26a771ff68495..990e9e6c83a29 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/PojoCsvInputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/PojoCsvInputFormat.java @@ -17,11 +17,11 @@ */ package org.apache.flink.api.java.io; -import com.google.common.base.Preconditions; import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.core.fs.FileInputSplit; import org.apache.flink.core.fs.Path; +import org.apache.flink.util.Preconditions; import java.io.IOException; import java.lang.reflect.Field; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/TextValueInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/TextValueInputFormat.java index fbfcd8d09aff5..a0d20d6ae4b9e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/TextValueInputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/TextValueInputFormat.java @@ -23,9 +23,9 @@ import java.nio.charset.CharacterCodingException; import java.nio.charset.Charset; import java.nio.charset.CharsetDecoder; +import java.nio.charset.StandardCharsets; import java.util.Arrays; -import com.google.common.base.Charsets; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.io.DelimitedInputFormat; import org.apache.flink.configuration.Configuration; @@ -85,7 +85,7 @@ public void configure(Configuration parameters) { throw new RuntimeException("Unsupported charset: " + charsetName); } - if (charsetName.equalsIgnoreCase(Charsets.US_ASCII.name())) { + if (charsetName.equalsIgnoreCase(StandardCharsets.US_ASCII.name())) { ascii = true; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index 20aabd998e833..26cf054525384 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -40,8 +40,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; - -import com.google.common.base.Preconditions; +import org.apache.flink.util.Preconditions; /** * This operator represents the application of a "aggregate" operation on a data set, and the diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index e69e16e9ae9a7..36e6c1c2a0916 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -36,12 +36,11 @@ import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.util.Preconditions; //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.java.tuple.*; -import com.google.common.base.Preconditions; - /** * A {@link DataSet} that is the result of a Cross transformation. * diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java index ccf88a6f8657b..d53b499a4b92d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java @@ -30,8 +30,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; - -import com.google.common.base.Preconditions; +import org.apache.flink.util.Preconditions; /** * The DeltaIteration represents the start of a delta iteration. It is created from the DataSet that diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index 1d1ec27928e38..88c479b83a4a2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -20,8 +20,6 @@ import java.util.Arrays; -import com.google.common.base.Preconditions; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; @@ -59,6 +57,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.java.tuple.*; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index d8a583549eaf2..6c27167059521 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -32,13 +32,12 @@ import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.translation.PlanProjectOperator; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.util.Preconditions; //CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.java.tuple.*; //CHECKSTYLE.ON: AvoidStarImport -import com.google.common.base.Preconditions; - /** * This operator represents the application of a projection operation on a data set, and the * result data set produced by the function. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index c4ae24bf00ef9..11645dfbeed6a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -35,8 +35,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.typeutils.TypeExtractor; - -import com.google.common.base.Preconditions; +import org.apache.flink.util.Preconditions; /** * SortedGrouping is an intermediate step for a transformation on a grouped and sorted DataSet.
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index ed5754c78c08e..2ad133c7b7646 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -36,8 +36,7 @@ import org.apache.flink.api.java.functions.SelectByMinFunction; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; - -import com.google.common.base.Preconditions; +import org.apache.flink.util.Preconditions; @Public public class UnsortedGrouping extends Grouping { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/CombineToGroupCombineWrapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/CombineToGroupCombineWrapper.java index 3488403013293..408d4b3098a08 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/CombineToGroupCombineWrapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/CombineToGroupCombineWrapper.java @@ -17,12 +17,12 @@ */ package org.apache.flink.api.java.operators.translation; -import com.google.common.base.Preconditions; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.CombineFunction; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; /** * A wrapper the wraps a function that implements both {@link CombineFunction} and {@link GroupReduceFunction} interfaces diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java index 680f36d069ea6..d8c54d6147c2a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java @@ -17,7 +17,6 @@ */ package org.apache.flink.api.java.operators.translation; -import com.google.common.base.Preconditions; import org.apache.flink.api.common.functions.CombineFunction; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; @@ -25,6 +24,7 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; /** * A wrapper the wraps a function that implements both {@link CombineFunction} and {@link GroupReduceFunction} interfaces diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java index cff5f1cdb66b6..c370f9d1f1136 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/BernoulliSampler.java @@ -17,8 +17,8 @@ */ package org.apache.flink.api.java.sampling; -import com.google.common.base.Preconditions; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.XORShiftRandom; import java.util.Iterator; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java index ae8ea57a67771..9b9828845f500 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/PoissonSampler.java @@ -17,9 +17,9 @@ */ package org.apache.flink.api.java.sampling; -import com.google.common.base.Preconditions; import org.apache.commons.math3.distribution.PoissonDistribution; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.XORShiftRandom; import java.util.Iterator; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java index 8a57279a10be2..b36d040a794d9 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithReplacement.java @@ -17,8 +17,8 @@ */ package org.apache.flink.api.java.sampling; -import com.google.common.base.Preconditions; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.XORShiftRandom; import java.util.Iterator; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java index 59161ab2e1f61..a38a921b0420b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/sampling/ReservoirSamplerWithoutReplacement.java @@ -17,8 +17,8 @@ */ package org.apache.flink.api.java.sampling; -import com.google.common.base.Preconditions; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.XORShiftRandom; import java.util.Iterator; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java index 692eea7cdd64a..1d65c7ea8ba24 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java @@ -18,7 +18,6 @@ package org.apache.flink.api.java.utils; -import com.google.common.collect.Lists; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.distributions.DataDistribution; @@ -48,6 +47,7 @@ import org.apache.flink.util.AbstractID; import org.apache.flink.util.Collector; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -104,7 +104,10 @@ public void open(Configuration parameters) throws Exception { @Override public List> initializeBroadcastVariable(Iterable> data) { // sort the list by task id to calculate the correct offset - List> sortedData = Lists.newArrayList(data); + List> sortedData = new ArrayList<>(); + for (Tuple2 datum : data) { + sortedData.add(datum); + } Collections.sort(sortedData, new Comparator>() { @Override public int compare(Tuple2 o1, Tuple2 o2) { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java index bfd6d12750c5f..6be78e2b50cba 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java @@ -17,13 +17,13 @@ */ package org.apache.flink.api.java.utils; -import com.google.common.base.Preconditions; import org.apache.commons.cli.Option; import org.apache.commons.lang3.math.NumberUtils; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.util.GenericOptionsParser; import java.io.File; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java index b89f6f539c82e..f44fe9e4e7393 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/io/CsvInputFormatTest.java @@ -19,8 +19,6 @@ package org.apache.flink.api.java.io; -import com.google.common.base.Charsets; - import org.apache.flink.api.common.io.ParseException; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.java.tuple.*; @@ -38,6 +36,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -692,7 +691,7 @@ private FileInputSplit createTempFile(String content) throws IOException { tempFile.deleteOnExit(); OutputStreamWriter wrt = new OutputStreamWriter( - new FileOutputStream(tempFile), Charsets.UTF_8 + new FileOutputStream(tempFile), StandardCharsets.UTF_8 ); wrt.write(content); wrt.close(); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java index 197fb2dbe209b..68f915494fb4d 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/sampling/RandomSamplerTest.java @@ -17,16 +17,15 @@ */ package org.apache.flink.api.java.sampling; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest; +import org.apache.flink.util.Preconditions; import org.junit.BeforeClass; import org.junit.Test; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.Iterator; import java.util.LinkedList; import java.util.List; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java b/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java index 8ac9c5901fb57..4420e99161fb7 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/tuple/TupleGenerator.java @@ -20,9 +20,9 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Scanner; -import com.google.common.base.Charsets; import com.google.common.io.Files; /** @@ -95,7 +95,7 @@ private static File getPackage(File root, String packageString) { } private static void insertCodeIntoFile(String code, File file) throws IOException { - String fileContent = Files.toString(file, Charsets.UTF_8); + String fileContent = Files.toString(file, StandardCharsets.UTF_8); try (Scanner s = new Scanner(fileContent)) { StringBuilder sb = new StringBuilder(); @@ -137,7 +137,7 @@ private static void insertCodeIntoFile(String code, File file) throws IOExceptio sb.append(line).append("\n"); } s.close(); - Files.write(sb.toString(), file, Charsets.UTF_8); + Files.write(sb.toString(), file, StandardCharsets.UTF_8); } } @@ -495,4 +495,4 @@ private static void appendTupleTypeGenerics(StringBuilder sb, int numFields) { "// THIS IS A GENERATED SOURCE FILE. DO NOT EDIT!\n" + "// GENERATED FROM " + TupleGenerator.class.getName() + ".\n" + "// --------------------------------------------------------------\n\n\n"; -} \ No newline at end of file +} From a974e321b66d5276769417918fbb219f4053cbab Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 4 May 2016 19:03:29 +0200 Subject: [PATCH 10/70] [hotfix] [tableAPI] Fix SQL queries on TableSources. --- .../api/table/StreamTableEnvironment.scala | 7 ++- .../datastream/StreamTableSourceScan.scala | 4 +- .../api/table/plan/rules/FlinkRuleSets.scala | 8 +-- .../plan/rules/LogicalScanToStreamable.scala | 56 ------------------- .../rules/datastream/RemoveDeltaRule.scala | 42 ++++++++++++++ .../StreamTableSourceScanRule.scala | 9 +-- .../schema/StreamableTableSourceTable.scala | 30 ++++++++++ .../table/plan/schema/TableSourceTable.scala | 4 +- .../table/plan/schema/TransStreamTable.scala | 5 -- .../api/scala/stream/TableSourceITCase.scala | 2 +- 10 files changed, 87 insertions(+), 80 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/LogicalScanToStreamable.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/RemoveDeltaRule.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/StreamableTableSourceTable.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala index 918a65fee50f3..be1c00569c4e4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala @@ -29,8 +29,9 @@ import org.apache.flink.api.table.expressions.Expression import org.apache.flink.api.table.plan.PlanGenException import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamRel, DataStreamConvention} import org.apache.flink.api.table.plan.rules.FlinkRuleSets -import org.apache.flink.api.table.plan.schema.{TableSourceTable, TransStreamTable, DataStreamTable} import org.apache.flink.api.table.sinks.{StreamTableSink, TableSink} +import org.apache.flink.api.table.plan.schema. + {StreamableTableSourceTable, TransStreamTable, DataStreamTable} import org.apache.flink.api.table.sources.StreamTableSource import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment @@ -110,7 +111,7 @@ abstract class StreamTableEnvironment( def registerTableSource(name: String, tableSource: StreamTableSource[_]): Unit = { checkValidTableName(name) - registerTableInternal(name, new TableSourceTable(tableSource)) + registerTableInternal(name, new StreamableTableSourceTable(tableSource)) } /** @@ -179,7 +180,7 @@ abstract class StreamTableEnvironment( fieldIndexes, fieldNames ) - // when registering a DataStream, we need to wrap it into a StreamableTable + // when registering a DataStream, we need to wrap it into a TransStreamTable // so that the SQL validation phase won't fail if (wrapper) { registerTableInternal(name, dataStreamTable) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala index 21b8a6310e6e1..2c7a5846cdc60 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala @@ -23,7 +23,7 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.RelDataType import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.table.StreamTableEnvironment -import org.apache.flink.api.table.plan.schema.TableSourceTable +import org.apache.flink.api.table.plan.schema.StreamableTableSourceTable import org.apache.flink.api.table.sources.StreamTableSource import org.apache.flink.streaming.api.datastream.DataStream @@ -35,7 +35,7 @@ class StreamTableSourceScan( rowType: RelDataType) extends StreamScan(cluster, traitSet, table, rowType) { - val tableSourceTable = table.unwrap(classOf[TableSourceTable]) + val tableSourceTable = table.unwrap(classOf[StreamableTableSourceTable]) val tableSource = tableSourceTable.tableSource.asInstanceOf[StreamTableSource[_]] override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala index 5d5912bed78c3..4ce8e5f165042 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala @@ -110,12 +110,10 @@ object FlinkRuleSets { /** * RuleSet to optimize plans for batch / DataSet execution */ - val DATASTREAM_OPT_RULES: RuleSet = { - - val rules = List( + val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList( + RemoveDeltaRule.INSTANCE, EnumerableToLogicalTableScan.INSTANCE, - LogicalScanToStreamable.INSTANCE, // calc rules FilterToCalcRule.INSTANCE, @@ -148,6 +146,4 @@ object FlinkRuleSets { StreamTableSourceScanRule.INSTANCE ) - RuleSets.ofList(rules ++ StreamRules.RULES.asList.take(7)) - } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/LogicalScanToStreamable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/LogicalScanToStreamable.scala deleted file mode 100644 index 3b389bccdf588..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/LogicalScanToStreamable.scala +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.table.plan.rules - -import org.apache.calcite.plan.RelOptRule._ -import org.apache.calcite.plan.{RelOptRuleCall, RelOptRule, RelOptRuleOperand} -import org.apache.calcite.prepare.RelOptTableImpl -import org.apache.calcite.rel.logical.LogicalTableScan -import org.apache.calcite.schema.StreamableTable -import org.apache.flink.api.table.plan.schema.TransStreamTable - -/** - * Custom rule that converts a LogicalScan into another LogicalScan - * whose internal Table is [[StreamableTable]] and [[org.apache.calcite.schema.TranslatableTable]]. - */ -class LogicalScanToStreamable( - operand: RelOptRuleOperand, - description: String) extends RelOptRule(operand, description) { - - override def onMatch(call: RelOptRuleCall): Unit = { - val oldRel = call.rel(0).asInstanceOf[LogicalTableScan] - val table = oldRel.getTable - table.unwrap(classOf[StreamableTable]) match { - case s: StreamableTable => - // already a StreamableTable => do nothing - case _ => // convert to a StreamableTable - val sTable = new TransStreamTable(oldRel, false) - val newRel = LogicalTableScan.create(oldRel.getCluster, - RelOptTableImpl.create(table.getRelOptSchema, table.getRowType, sTable)) - call.transformTo(newRel) - } - } -} - -object LogicalScanToStreamable { - val INSTANCE = new LogicalScanToStreamable( - operand(classOf[LogicalTableScan], any), - "LogicalScanToStreamable") -} - diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/RemoveDeltaRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/RemoveDeltaRule.scala new file mode 100644 index 0000000000000..7b4720a4b801a --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/RemoveDeltaRule.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.plan.rules.datastream + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.stream.LogicalDelta + +/** + * Rule that converts an EnumerableTableScan into a LogicalTableScan. + * We need this rule because Calcite creates an EnumerableTableScan + * when parsing a SQL query. We convert it into a LogicalTableScan + * so we can merge the optimization process with any plan that might be created + * by the Table API. + */ +class RemoveDeltaRule extends RelOptRule(operand(classOf[LogicalDelta], any), "RemoveDeltaRule") { + + override def onMatch(call: RelOptRuleCall): Unit = { + val delta = call.rel(0).asInstanceOf[LogicalDelta] + call.transformTo(delta.getInput) + } +} + +object RemoveDeltaRule { + val INSTANCE = new RemoveDeltaRule() +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala index 5ecf9941977a7..8000cdecb56b8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala @@ -23,8 +23,9 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan import org.apache.calcite.rel.logical.LogicalTableScan -import org.apache.flink.api.table.plan.nodes.datastream.{StreamTableSourceScan, DataStreamConvention} -import org.apache.flink.api.table.plan.schema.TableSourceTable +import org.apache.flink.api.table.plan.nodes.datastream. + {StreamTableSourceScan, DataStreamConvention} +import org.apache.flink.api.table.plan.schema.StreamableTableSourceTable import org.apache.flink.api.table.sources.StreamTableSource /** Rule to convert a [[LogicalTableScan]] into a [[StreamTableSourceScan]]. */ @@ -39,9 +40,9 @@ class StreamTableSourceScanRule /** Rule must only match if TableScan targets a [[StreamTableSource]] */ override def matches(call: RelOptRuleCall): Boolean = { val scan: TableScan = call.rel(0).asInstanceOf[TableScan] - val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable]) + val dataSetTable = scan.getTable.unwrap(classOf[StreamableTableSourceTable]) dataSetTable match { - case tst: TableSourceTable => + case tst: StreamableTableSourceTable => tst.tableSource match { case _: StreamTableSource[_] => true diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/StreamableTableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/StreamableTableSourceTable.scala new file mode 100644 index 0000000000000..58214bcbd4c56 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/StreamableTableSourceTable.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.plan.schema + +import org.apache.calcite.schema.{Table, StreamableTable} +import org.apache.flink.api.table.sources.TableSource + +/** Table which defines an external streamable table via a [[TableSource]] */ +class StreamableTableSourceTable(tableSource: TableSource[_]) + extends TableSourceTable(tableSource) + with StreamableTable { + + override def stream(): Table = new TableSourceTable(tableSource) +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala index 042c82310779b..03646f90aff37 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala @@ -27,6 +27,4 @@ class TableSourceTable(val tableSource: TableSource[_]) extends FlinkTable[Row]( typeInfo = new RowTypeInfo(tableSource.getFieldTypes, tableSource.getFieldsNames), fieldIndexes = 0.until(tableSource.getNumberOfFields).toArray, - fieldNames = tableSource.getFieldsNames.toArray) { - -} + fieldNames = tableSource.getFieldsNames) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TransStreamTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TransStreamTable.scala index bc2765917c5f0..61f2598f35542 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TransStreamTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TransStreamTable.scala @@ -33,11 +33,6 @@ import org.apache.calcite.schema.{StreamableTable, Table, TranslatableTable} * can be converted to a relational expression and [[StreamableTable]] * so that it can be used in Streaming SQL queries. * - * Except for registering Streaming Tables, this implementation is also used - * in [[org.apache.flink.api.table.plan.rules.LogicalScanToStreamable]] - * rule to convert a logical scan of a non-Streamable Table into - * a logical scan of a Streamable table, i.e. of this class. - * * @see [[DataStreamTable]] */ class TransStreamTable(relNode: RelNode, wrapper: Boolean) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala index 687cc2874937b..50d9a4245f09d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala @@ -70,7 +70,7 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase { tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33)) tEnv.sql( - "SELECT amount * id, name FROM MyTestTable WHERE amount < 4") + "SELECT STREAM amount * id, name FROM MyTestTable WHERE amount < 4") .toDataStream[Row] .addSink(new StreamITCase.StringSink) From f2fa73faf279869a1295edcff0c5edd9dd80ccf3 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 28 Apr 2016 15:03:22 +0200 Subject: [PATCH 11/70] [FLINK-3842] [tableApi] Fix handling null record/row in generated code This closes #1974 --- .../api/table/codegen/CodeGenerator.scala | 189 ++++++++++++------ .../table/plan/nodes/dataset/BatchScan.scala | 4 +- .../plan/nodes/dataset/DataSetAggregate.scala | 13 +- .../plan/nodes/dataset/DataSetCalc.scala | 2 +- .../plan/nodes/dataset/DataSetJoin.scala | 6 +- .../table/plan/nodes/dataset/DataSetRel.scala | 4 +- .../plan/nodes/dataset/DataSetSort.scala | 5 +- .../nodes/datastream/DataStreamCalc.scala | 2 +- .../plan/nodes/datastream/StreamScan.scala | 1 + .../utils/ExpressionEvaluator.scala | 2 +- 10 files changed, 150 insertions(+), 78 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala index e090a292e2e3f..c8d6dca5ec55f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala @@ -42,6 +42,7 @@ import scala.collection.mutable * A code generator for generating Flink [[org.apache.flink.api.common.functions.Function]]s. * * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. * @param input1 type information about the first input of the Function * @param input2 type information about the second input if the Function is binary * @param inputPojoFieldMapping additional mapping information if input1 is a POJO (POJO types @@ -50,11 +51,17 @@ import scala.collection.mutable */ class CodeGenerator( config: TableConfig, + nullableInput: Boolean, input1: TypeInformation[Any], input2: Option[TypeInformation[Any]] = None, inputPojoFieldMapping: Option[Array[Int]] = None) extends RexVisitor[GeneratedExpression] { + // check if nullCheck is enabled when inputs can be null + if (nullableInput && !config.getNullCheck) { + throw new CodeGenException("Null check must be enabled if entire rows can be null.") + } + // check for POJO input mapping input1 match { case pt: PojoTypeInfo[_] => @@ -65,7 +72,7 @@ class CodeGenerator( // check that input2 is never a POJO input2 match { - case pt: PojoTypeInfo[_] => + case Some(pt: PojoTypeInfo[_]) => throw new CodeGenException("Second input must not be a POJO type.") case _ => // ok } @@ -75,12 +82,17 @@ class CodeGenerator( * [[org.apache.flink.api.common.functions.Function]]s with one input. * * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. * @param input type information about the input of the Function * @param inputPojoFieldMapping additional mapping information necessary if input is a * POJO (POJO types have no deterministic field order). */ - def this(config: TableConfig, input: TypeInformation[Any], inputPojoFieldMapping: Array[Int]) = - this(config, input, None, Some(inputPojoFieldMapping)) + def this( + config: TableConfig, + nullableInput: Boolean, + input: TypeInformation[Any], + inputPojoFieldMapping: Array[Int]) = + this(config, nullableInput, input, None, Some(inputPojoFieldMapping)) // set of member statements that will be added only once @@ -212,7 +224,7 @@ class CodeGenerator( ${reuseMemberCode()} - public $funcName() throws Exception{ + public $funcName() throws Exception { ${reuseInitCode()} } @@ -785,73 +797,128 @@ class CodeGenerator( // generate input access and boxing if necessary case None => - val newExpr = inputType match { + val expr = if (nullableInput) { + generateNullableInputFieldAccess(inputType, inputTerm, index) + } + else { + generateFieldAccess(inputType, inputTerm, index) + } - case ct: CompositeType[_] => - val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) { - inputPojoFieldMapping.get(index) - } - else { - index - } - val accessor = fieldAccessorFor(ct, fieldIndex) - val fieldType: TypeInformation[Any] = ct.getTypeAt(fieldIndex) - val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType) + reusableInputUnboxingExprs((inputTerm, index)) = expr + expr + } + // hide the generated code as it will be executed only once + GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType) + } - accessor match { - case ObjectFieldAccessor(field) => - // primitive - if (isFieldPrimitive(field)) { - generateNonNullLiteral(fieldType, s"$inputTerm.${field.getName}") - } - // Object - else { - generateNullableLiteral( - fieldType, - s"($fieldTypeTerm) $inputTerm.${field.getName}") - } + private def generateNullableInputFieldAccess( + inputType: TypeInformation[Any], + inputTerm: String, + index: Int) + : GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val fieldType = inputType match { + case ct: CompositeType[_] => + val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) { + inputPojoFieldMapping.get(index) + } + else { + index + } + ct.getTypeAt(fieldIndex) + case at: AtomicType[_] => at + case _ => throw new CodeGenException("Unsupported type for input field access.") + } + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + val defaultValue = primitiveDefaultValue(fieldType) + val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index) - case ObjectGenericFieldAccessor(fieldName) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName" - generateNullableLiteral(fieldType, inputCode) + val inputCheckCode = + s""" + |$resultTypeTerm $resultTerm; + |boolean $nullTerm; + |if ($inputTerm == null) { + | $resultTerm = $defaultValue; + | $nullTerm = true; + |} + |else { + | ${fieldAccessExpr.code} + | $resultTerm = ${fieldAccessExpr.resultTerm}; + | $nullTerm = ${fieldAccessExpr.nullTerm}; + |} + |""".stripMargin - case ObjectMethodAccessor(methodName) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()" - generateNullableLiteral(fieldType, inputCode) + GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType) + } - case ProductAccessor(i) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.productElement($i)" - generateNullableLiteral(fieldType, inputCode) + private def generateFieldAccess( + inputType: TypeInformation[Any], + inputTerm: String, + index: Int) + : GeneratedExpression = { + inputType match { + case ct: CompositeType[_] => + val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) { + inputPojoFieldMapping.get(index) + } + else { + index + } + val accessor = fieldAccessorFor(ct, fieldIndex) + val fieldType: TypeInformation[Any] = ct.getTypeAt(fieldIndex) + val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType) + + accessor match { + case ObjectFieldAccessor(field) => + // primitive + if (isFieldPrimitive(field)) { + generateNonNullLiteral(fieldType, s"$inputTerm.${field.getName}") + } + // Object + else { + generateNullableLiteral( + fieldType, + s"($fieldTypeTerm) $inputTerm.${field.getName}") + } - case ObjectPrivateFieldAccessor(field) => - val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName) - val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm) - // primitive - if (isFieldPrimitive(field)) { - generateNonNullLiteral(fieldType, reflectiveAccessCode) - } - // Object - else { - generateNullableLiteral(fieldType, reflectiveAccessCode) - } + case ObjectGenericFieldAccessor(fieldName) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName" + generateNullableLiteral(fieldType, inputCode) + + case ObjectMethodAccessor(methodName) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()" + generateNullableLiteral(fieldType, inputCode) + + case ProductAccessor(i) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.productElement($i)" + generateNullableLiteral(fieldType, inputCode) + + case ObjectPrivateFieldAccessor(field) => + val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName) + val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm) + // primitive + if (isFieldPrimitive(field)) { + generateNonNullLiteral(fieldType, reflectiveAccessCode) } + // Object + else { + generateNullableLiteral(fieldType, reflectiveAccessCode) + } + } - case at: AtomicType[_] => - val fieldTypeTerm = boxedTypeTermForTypeInfo(at) - val inputCode = s"($fieldTypeTerm) $inputTerm" - generateNullableLiteral(at, inputCode) + case at: AtomicType[_] => + val fieldTypeTerm = boxedTypeTermForTypeInfo(at) + val inputCode = s"($fieldTypeTerm) $inputTerm" + generateNullableLiteral(at, inputCode) - case _ => - throw new CodeGenException("Unsupported type for input access.") - } - reusableInputUnboxingExprs((inputTerm, index)) = newExpr - newExpr + case _ => + throw new CodeGenException("Unsupported type for input field access.") } - // hide the generated code as it will be executed only once - GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType) } private def generateNullableLiteral( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala index b18d6747be444..85ed6ef4cc52a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala @@ -22,14 +22,11 @@ import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan import org.apache.calcite.rel.metadata.RelMetadataQuery -import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet import org.apache.flink.api.java.typeutils.PojoTypeInfo import org.apache.flink.api.table.TableConfig -import org.apache.flink.api.table.codegen.CodeGenerator import org.apache.flink.api.table.plan.schema.FlinkTable -import org.apache.flink.api.table.runtime.MapRunner import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType import scala.collection.JavaConversions._ @@ -84,6 +81,7 @@ abstract class BatchScan( val mapFunc = getConversionMapper( config, + false, inputType, determinedType, "DataSetSourceConversion", diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala index 114122bd020aa..910f05c9c5549 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala @@ -18,20 +18,17 @@ package org.apache.flink.api.table.plan.nodes.dataset -import org.apache.calcite.plan.{RelOptCost, RelOptPlanner, RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} -import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet -import org.apache.flink.api.table.codegen.CodeGenerator -import org.apache.flink.api.table.runtime.MapRunner import org.apache.flink.api.table.runtime.aggregate.AggregateUtil import org.apache.flink.api.table.runtime.aggregate.AggregateUtil.CalcitePair -import org.apache.flink.api.table.typeutils.{TypeConverter, RowTypeInfo} -import org.apache.flink.api.table.{BatchTableEnvironment, Row, TableConfig} +import org.apache.flink.api.table.typeutils.{RowTypeInfo, TypeConverter} +import org.apache.flink.api.table.{BatchTableEnvironment, Row} import scala.collection.JavaConverters._ @@ -143,7 +140,9 @@ class DataSetAggregate( expectedType match { case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] => val mapName = s"convert: (${rowType.getFieldNames.asScala.toList.mkString(", ")})" - result.map(getConversionMapper(config, + result.map(getConversionMapper( + config, + false, rowTypeInfo.asInstanceOf[TypeInformation[Any]], expectedType.get, "AggregateOutputConversion", diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala index 13bb39d3da0fe..75e4fd236e306 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala @@ -103,7 +103,7 @@ class DataSetCalc( config.getNullCheck, config.getEfficientTypeUsage) - val generator = new CodeGenerator(config, inputDS.getType) + val generator = new CodeGenerator(config, false, inputDS.getType) val body = functionBody( generator, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala index 61e8995f11677..4f24f4e18a6ec 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala @@ -159,7 +159,11 @@ class DataSetJoin( val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv) val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv) - val generator = new CodeGenerator(config, leftDataSet.getType, Some(rightDataSet.getType)) + val generator = new CodeGenerator( + config, + false, + leftDataSet.getType, + Some(rightDataSet.getType)) val conversion = generator.generateConverterResultExpression( returnType, joinRowType.getFieldNames) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala index 7c76e4692f0cf..8c14e9e56238c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala @@ -25,9 +25,9 @@ import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet import org.apache.flink.api.table.codegen.CodeGenerator -import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig, TableEnvironment} import org.apache.flink.api.table.plan.nodes.FlinkRel import org.apache.flink.api.table.runtime.MapRunner +import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig} import scala.collection.JavaConversions._ @@ -69,6 +69,7 @@ trait DataSetRel extends RelNode with FlinkRel { private[dataset] def getConversionMapper( config: TableConfig, + nullableInput: Boolean, inputType: TypeInformation[Any], expectedType: TypeInformation[Any], conversionOperatorName: String, @@ -77,6 +78,7 @@ trait DataSetRel extends RelNode with FlinkRel { val generator = new CodeGenerator( config, + nullableInput, inputType, None, inputPojoFieldMapping) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala index ef89b06a9e6cb..23cfbcf619ed6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala @@ -27,7 +27,6 @@ import org.apache.calcite.rel.{RelCollation, RelNode, RelWriter, SingleRel} import org.apache.flink.api.common.operators.Order import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet -import org.apache.flink.api.java.typeutils.PojoTypeInfo import org.apache.flink.api.table.BatchTableEnvironment import org.apache.flink.api.table.typeutils.TypeConverter._ @@ -88,7 +87,9 @@ class DataSetSort( // conversion if (determinedType != inputType) { - val mapFunc = getConversionMapper(config, + val mapFunc = getConversionMapper( + config, + false, partitionedDs.getType, determinedType, "DataSetSortConversion", diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala index 6dfcd03ad6754..334c0aafb0a02 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala @@ -82,7 +82,7 @@ class DataStreamCalc( config.getNullCheck, config.getEfficientTypeUsage) - val generator = new CodeGenerator(config, inputDataStream.getType) + val generator = new CodeGenerator(config, false, inputDataStream.getType) val body = functionBody( generator, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala index 7ee9a912f88d3..1f5fc6a41d683 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala @@ -72,6 +72,7 @@ abstract class StreamScan( if (determinedType != inputType) { val generator = new CodeGenerator( config, + false, input.getType, flinkTable.fieldIndexes) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala index e0316c2b9b3bc..56daa6bf76b67 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala @@ -93,7 +93,7 @@ object ExpressionEvaluator { rexNode: RexNode): String = { // generate code for Mapper val config = new TableConfig() - val generator = new CodeGenerator(config, typeInfo) + val generator = new CodeGenerator(config, false, typeInfo) val genExpr = generator.generateExpression(relBuilder.cast(rexNode, VARCHAR)) // cast to String val bodyCode = s""" From 43272f5117a908e53a5ee55e4742c78fe18caf6b Mon Sep 17 00:00:00 2001 From: markreddy Date: Sun, 8 May 2016 21:41:13 +0100 Subject: [PATCH 12/70] [FLINK-3882] [docs] Fix errors in sample Java code for the Elasticsearch2 sink This closes #1971 --- .../apis/streaming/connectors/elasticsearch2.md | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/apis/streaming/connectors/elasticsearch2.md b/docs/apis/streaming/connectors/elasticsearch2.md index 74650ec8afb22..7146285925968 100644 --- a/docs/apis/streaming/connectors/elasticsearch2.md +++ b/docs/apis/streaming/connectors/elasticsearch2.md @@ -78,20 +78,19 @@ transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); transports.add(new InetSocketAddress(InetAddress.getByName("10.2.3.1"), 9300)); input.addSink(new ElasticsearchSink(config, transports, new ElasticsearchSinkFunction() { - public IndexRequest createIndexRequest(String element): IndexRequest = { - Map json = new HashMap<>() - json.put("data", element) + public IndexRequest createIndexRequest(String element) { + Map json = new HashMap<>(); + json.put("data", element); return Requests.indexRequest() .index("my-index") .type("my-type") .source(json); - } - + @Override public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { - indexer.add(createIndexRequest(element)) + indexer.add(createIndexRequest(element)); } })); {% endhighlight %} @@ -116,10 +115,10 @@ input.addSink(new ElasticsearchSink(config, transports, new ElasticsearchSinkFun json.put("data", element) Requests.indexRequest.index("my-index").`type`("my-type").source(json) } - + override def process(element: String, ctx: RuntimeContext, indexer: RequestIndexer) { indexer.add(createIndexRequest(element)) - } + } })) {% endhighlight %} @@ -139,7 +138,7 @@ This will buffer elements and Action Requests before sending to the cluster. The * **bulk.flush.interval.ms**: Interval at which to flush data regardless of the other two settings in milliseconds -This now provides a list of Elasticsearch Nodes +This now provides a list of Elasticsearch Nodes to which the sink should connect via a `TransportClient`. More about information about Elasticsearch can be found [here](https://elastic.co). From b1cff6baecb9122549cc7d7071da11e7f3951ba4 Mon Sep 17 00:00:00 2001 From: twalthr Date: Mon, 2 May 2016 16:31:45 +0200 Subject: [PATCH 13/70] [FLINK-3856] [core] [api-extending] Create types for java.sql.Date/Time/Timestamp This closes #1959 --- .../api/common/typeinfo/SqlTimeTypeInfo.java | 168 ++++++++++++++++++ .../common/typeutils/base/DateComparator.java | 40 +++-- .../common/typeutils/base/DateSerializer.java | 22 ++- .../typeutils/base/SqlDateSerializer.java | 105 +++++++++++ .../typeutils/base/SqlTimeSerializer.java | 104 +++++++++++ .../base/SqlTimestampComparator.java | 98 ++++++++++ .../base/SqlTimestampSerializer.java | 113 ++++++++++++ .../api/java/typeutils/TypeExtractor.java | 21 +++ .../typeutils/base/SqlDateComparatorTest.java | 49 +++++ .../typeutils/base/SqlDateSerializerTest.java | 55 ++++++ .../typeutils/base/SqlTimeComparatorTest.java | 48 +++++ .../typeutils/base/SqlTimeSerializerTest.java | 55 ++++++ .../base/SqlTimestampComparatorTest.java | 51 ++++++ .../base/SqlTimestampSerializerTest.java | 58 ++++++ .../api/java/typeutils/TypeExtractorTest.java | 36 ++++ 15 files changed, 996 insertions(+), 27 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfo.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/SqlDateSerializer.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/SqlTimeSerializer.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/SqlTimestampComparator.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/SqlTimestampSerializer.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateComparatorTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateSerializerTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlTimeComparatorTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlTimeSerializerTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlTimestampComparatorTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlTimestampSerializerTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfo.java new file mode 100644 index 0000000000000..a05227c6b395b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfo.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeinfo; + +import java.lang.reflect.Constructor; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Objects; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.DateComparator; +import org.apache.flink.api.common.typeutils.base.SqlDateSerializer; +import org.apache.flink.api.common.typeutils.base.SqlTimeSerializer; +import org.apache.flink.api.common.typeutils.base.SqlTimestampComparator; +import org.apache.flink.api.common.typeutils.base.SqlTimestampSerializer; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Type information for Java SQL Date/Time/Timestamp. + */ +@PublicEvolving +public class SqlTimeTypeInfo extends TypeInformation implements AtomicType { + + private static final long serialVersionUID = -132955295409131880L; + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static final SqlTimeTypeInfo DATE = new SqlTimeTypeInfo<>(Date.class, SqlDateSerializer.INSTANCE, (Class) DateComparator.class); + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static final SqlTimeTypeInfo

IMPORTANT: This constructor may not start any work that would need to @@ -253,7 +253,11 @@ public Task(TaskDeploymentDescriptor tdd, this.nameOfInvokableClass = checkNotNull(tdd.getInvokableClassName()); this.operatorState = tdd.getOperatorState(); this.recoveryTs = tdd.getRecoveryTimestamp(); - this.executionConfig = checkNotNull(tdd.getExecutionConfig()); + this.serializedExecutionConfig = checkNotNull(tdd.getSerializedExecutionConfig()); + + this.taskCancellationInterval = jobConfiguration.getLong( + ConfigConstants.TASK_CANCELLATION_INTERVAL_MILLIS, + ConfigConstants.DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS); this.memoryManager = checkNotNull(memManager); this.ioManager = checkNotNull(ioManager); @@ -271,15 +275,6 @@ public Task(TaskDeploymentDescriptor tdd, this.executionListenerActors = new CopyOnWriteArrayList(); - if (executionConfig.getTaskCancellationInterval() < 0) { - taskCancellationInterval = jobConfiguration.getLong( - ConfigConstants.TASK_CANCELLATION_INTERVAL_MILLIS, - ConfigConstants.DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS); - } else { - taskCancellationInterval = executionConfig.getTaskCancellationInterval(); - } - - // create the reader and writer structures final String taskNameWithSubtaskAndId = taskNameWithSubtask + " (" + executionId + ')'; @@ -467,9 +462,14 @@ else if (current == ExecutionState.CANCELING) { // first of all, get a user-code classloader // this may involve downloading the job's JAR files and/or classes LOG.info("Loading JAR files for task " + taskNameWithSubtask); + final ClassLoader userCodeClassLoader = createUserCodeClassloader(libraryCache); + final ExecutionConfig executionConfig = serializedExecutionConfig.deserializeValue(userCodeClassLoader); - executionConfig.deserializeUserCode(userCodeClassLoader); + if (executionConfig.getTaskCancellationInterval() >= 0) { + // override task cancellation interval from Flink config if set in ExecutionConfig + taskCancellationInterval = executionConfig.getTaskCancellationInterval(); + } // now load the task's invokable code invokable = loadAndInstantiateInvokable(userCodeClassLoader, nameOfInvokableClass); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index d8b8a01e34614..3c633f3ad8b1d 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -28,7 +28,6 @@ import akka.actor._ import akka.pattern.ask import grizzled.slf4j.Logger -import org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} @@ -46,7 +45,7 @@ import org.apache.flink.runtime.clusterframework.messages._ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager -import org.apache.flink.runtime.executiongraph.restart.{RestartStrategy, RestartStrategyFactory} +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex} import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager} import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator @@ -1069,11 +1068,14 @@ class JobManager( throw new JobSubmissionException(jobId, "The given job is empty") } - val restartStrategy = Option(jobGraph.getExecutionConfig().getRestartStrategy()) - .map(RestartStrategyFactory.createRestartStrategy(_)) match { - case Some(strategy) => strategy - case None => restartStrategyFactory.createRestartStrategy() - } + val restartStrategy = + Option(jobGraph.getSerializedExecutionConfig() + .deserializeValue(userCodeLoader) + .getRestartStrategy()) + .map(RestartStrategyFactory.createRestartStrategy(_)) match { + case Some(strategy) => strategy + case None => restartStrategyFactory.createRestartStrategy() + } log.info(s"Using restart strategy $restartStrategy for $jobId.") @@ -1088,7 +1090,7 @@ class JobManager( jobGraph.getJobID, jobGraph.getName, jobGraph.getJobConfiguration, - jobGraph.getExecutionConfig, + jobGraph.getSerializedExecutionConfig, timeout, restartStrategy, jobGraph.getUserJarBlobKeys, @@ -1197,12 +1199,13 @@ class JobManager( new SimpleCheckpointStatsTracker(historySize, ackVertices) } - val jobParallelism = jobGraph.getExecutionConfig.getParallelism() + val jobParallelism = jobGraph.getSerializedExecutionConfig + .deserializeValue(userCodeLoader).getParallelism() val parallelism = if (jobParallelism == ExecutionConfig.PARALLELISM_AUTO_MAX) { numSlots } else { - jobGraph.getExecutionConfig.getParallelism + jobParallelism } executionGraph.enableSnapshotCheckpointing( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java index 03ff83d536b9a..91a83b257c4a3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java @@ -61,7 +61,7 @@ public void testCoordinatorShutsDownOnFailure() { vertex.setInvokableClass(Tasks.NoOpInvokable.class); List vertexIdList = Collections.singletonList(vertex.getID()); - JobGraph testGraph = new JobGraph("test job", new ExecutionConfig(), vertex); + JobGraph testGraph = new JobGraph("test job", vertex); testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 5000, 60000, 0L, Integer.MAX_VALUE)); @@ -113,7 +113,7 @@ public void testCoordinatorShutsDownOnSuccess() { vertex.setInvokableClass(Tasks.NoOpInvokable.class); List vertexIdList = Collections.singletonList(vertex.getID()); - JobGraph testGraph = new JobGraph("test job", new ExecutionConfig(), vertex); + JobGraph testGraph = new JobGraph("test job", vertex); testGraph.setSnapshotSettings(new JobSnapshottingSettings(vertexIdList, vertexIdList, vertexIdList, 5000, 60000, 0L, Integer.MAX_VALUE)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index 965556f268f3e..a801348bb1297 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.checkpoint; import akka.actor.ActorSystem; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -50,7 +50,7 @@ public void testCheckpointAndSavepointCoordinatorShareCheckpointIDCounter() thro new JobID(), "test", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), new FiniteDuration(1, TimeUnit.DAYS), new NoRestartStrategy(), Collections.emptyList(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java index 865760e686c5d..cc1994af12647 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java @@ -95,7 +95,7 @@ public void testJobClientRecovery() throws Exception { JobVertex blockingVertex = new JobVertex("Blocking Vertex"); blockingVertex.setInvokableClass(BlockingTask.class); blockingVertex.setParallelism(1); - final JobGraph jobGraph = new JobGraph("Blocking Test Job", new ExecutionConfig(), blockingVertex); + final JobGraph jobGraph = new JobGraph("Blocking Test Job", blockingVertex); final Promise promise = new scala.concurrent.impl.Promise.DefaultPromise<>(); Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java index ee1fd60353302..073164c030a41 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java @@ -48,7 +48,7 @@ public class JobClientActorTest extends TestLogger { private static ActorSystem system; - private static JobGraph testJobGraph = new JobGraph("Test Job", new ExecutionConfig()); + private static JobGraph testJobGraph = new JobGraph("Test Job"); @BeforeClass public static void setup() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 63e62bf0da9c0..36744a91815f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -27,6 +27,7 @@ import java.util.List; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -35,6 +36,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.BatchTask; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.util.SerializedValue; import org.junit.Test; public class TaskDeploymentDescriptorTest { @@ -55,7 +57,7 @@ public void testSerialization() { final List inputGates = new ArrayList(0); final List requiredJars = new ArrayList(0); final List requiredClasspaths = new ArrayList(0); - final ExecutionConfig executionConfig = new ExecutionConfig(); + final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, executionConfig, taskName, indexInSubtaskGroup, currentNumberOfSubtasks, attemptNumber, @@ -78,9 +80,7 @@ public void testSerialization() { assertEquals(orig.getAttemptNumber(), copy.getAttemptNumber()); assertEquals(orig.getProducedPartitions(), copy.getProducedPartitions()); assertEquals(orig.getInputGates(), copy.getInputGates()); - // load serialized values in ExecutionConfig - copy.getExecutionConfig().deserializeUserCode(getClass().getClassLoader()); - assertEquals(orig.getExecutionConfig(), copy.getExecutionConfig()); + assertEquals(orig.getSerializedExecutionConfig(), copy.getSerializedExecutionConfig()); assertEquals(orig.getRequiredJarFiles(), copy.getRequiredJarFiles()); assertEquals(orig.getRequiredClasspaths(), copy.getRequiredClasspaths()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java index d845d0168e35b..8eebe6618a89d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java @@ -26,7 +26,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -107,7 +107,7 @@ public void testCreateSimpleGraphBipartite() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -152,7 +152,7 @@ public void testAttachViaDataSets() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -220,7 +220,7 @@ public void testAttachViaIds() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -475,7 +475,7 @@ public void testCannotConnectMissingId() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -532,7 +532,7 @@ public void testCannotConnectWrongOrder() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -594,7 +594,7 @@ public void testSetupInputSplits() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -640,7 +640,7 @@ public void testMoreThanOneConsumerForIntermediateResult() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -705,14 +705,14 @@ public void testCoLocationConstraintCreation() { JobVertex v8 = new JobVertex("vertex8"); v8.setParallelism(2); - JobGraph jg = new JobGraph(jobId, jobName, new ExecutionConfig(), v1, v2, v3, v4, v5, v6, v7, v8); + JobGraph jg = new JobGraph(jobId, jobName, v1, v2, v3, v4, v5, v6, v7, v8); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index 7a9cee75b7f75..d126acb25fb8c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -84,8 +84,8 @@ public void testBuildDeploymentDescriptor() { TestingUtils.defaultExecutionContext(), jobId, "some job", - new Configuration(), - new ExecutionConfig(), + new Configuration(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -289,7 +289,7 @@ private Map setupExecution(JobVertex v1, int dop1 jobId, "some job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -332,4 +332,4 @@ public void finalizeOnMaster(ClassLoader cl) throws Exception { throw new Exception(); } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 0837927c89480..01cca5c93166d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.executiongraph; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.configuration.Configuration; @@ -73,14 +74,14 @@ public void testNoManualRestart() throws Exception { sender.setInvokableClass(Tasks.NoOpInvokable.class); sender.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender); + JobGraph jobGraph = new JobGraph("Pointwise job", sender); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -129,13 +130,13 @@ public void testConstraintsAfterRestart() throws Exception { groupVertex.setStrictlyCoLocatedWith(groupVertex2); //initiate and schedule job - JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), groupVertex, groupVertex2); + JobGraph jobGraph = new JobGraph("Pointwise job", groupVertex, groupVertex2); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 0L)); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -184,14 +185,14 @@ public void testRestartAutomatically() throws Exception { sender.setInvokableClass(Tasks.NoOpInvokable.class); sender.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender); + JobGraph jobGraph = new JobGraph("Pointwise job", sender); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "Test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000)); eg.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -220,7 +221,7 @@ public void testCancelWhileRestarting() throws Exception { new JobID(), "TestJob", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), // We want to manually control the restart and delay new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE)); @@ -229,7 +230,7 @@ public void testCancelWhileRestarting() throws Exception { jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); jobVertex.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("TestJob", new ExecutionConfig(), jobVertex); + JobGraph jobGraph = new JobGraph("TestJob", jobVertex); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -279,7 +280,7 @@ public void testCancelWhileFailing() throws Exception { new JobID(), "TestJob", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), // We want to manually control the restart and delay new FixedDelayRestartStrategy(Integer.MAX_VALUE, Long.MAX_VALUE)); @@ -295,7 +296,7 @@ public void testCancelWhileFailing() throws Exception { jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); jobVertex.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("TestJob", new ExecutionConfig(), jobVertex); + JobGraph jobGraph = new JobGraph("TestJob", jobVertex); executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); @@ -355,14 +356,14 @@ public void testNoRestartOnSuppressException() throws Exception { sender.setInvokableClass(Tasks.NoOpInvokable.class); sender.setParallelism(NUM_TASKS); - JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender); + JobGraph jobGraph = new JobGraph("Pointwise job", sender); ExecutionGraph eg = spy(new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "Test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000))); @@ -426,14 +427,14 @@ public void testFailingExecutionAfterRestart() throws Exception { receiver.setInvokableClass(Tasks.NoOpInvokable.class); receiver.setParallelism(1); - JobGraph jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver); + JobGraph jobGraph = new JobGraph("Pointwise job", sender, receiver); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000)); @@ -518,16 +519,18 @@ public void testFailExecutionAfterCancel() throws Exception { vertex.setInvokableClass(Tasks.NoOpInvokable.class); vertex.setParallelism(1); - JobGraph jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex); - jobGraph.getExecutionConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart( - Integer.MAX_VALUE, Integer.MAX_VALUE)); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart( + Integer.MAX_VALUE, Integer.MAX_VALUE)); + JobGraph jobGraph = new JobGraph("Test Job", vertex); + jobGraph.setExecutionConfig(executionConfig); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000000)); @@ -570,16 +573,18 @@ public void testFailExecutionGraphAfterCancel() throws Exception { vertex.setInvokableClass(Tasks.NoOpInvokable.class); vertex.setParallelism(1); - JobGraph jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex); - jobGraph.getExecutionConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart( - Integer.MAX_VALUE, Integer.MAX_VALUE)); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart( + Integer.MAX_VALUE, Integer.MAX_VALUE)); + JobGraph jobGraph = new JobGraph("Test Job", vertex); + jobGraph.setExecutionConfig(executionConfig); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new FixedDelayRestartStrategy(1, 1000000)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java index d1bb6801ee7b8..8b04fa39e9b11 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java @@ -22,7 +22,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.StoppingException; @@ -128,7 +128,7 @@ public void prepare() throws Exception { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(ordered); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 6659b5aa27b7f..92a74022e52ab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -22,10 +22,11 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import java.io.IOException; import java.lang.reflect.Field; import java.net.InetAddress; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.akka.AkkaUtils; @@ -176,7 +177,7 @@ public static ExecutionJobVertex getExecutionVertex(JobVertexID id, ExecutionCon new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); @@ -197,7 +198,7 @@ public Void answer(InvocationOnMock invocation) { return ejv; } - public static ExecutionJobVertex getExecutionVertex(JobVertexID id) throws JobException { + public static ExecutionJobVertex getExecutionVertex(JobVertexID id) throws JobException, IOException { return getExecutionVertex(id, TestingUtils.defaultExecutionContext()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java index 1ff90e13ac981..9e4aa6d82926f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java @@ -1,88 +1,88 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.executiongraph; - -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*; -import static org.junit.Assert.*; -import static org.mockito.Mockito.mock; - -import java.util.Arrays; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.junit.Test; - -public class ExecutionStateProgressTest { - - @Test - public void testAccumulatedStateFinished() { - try { - final JobID jid = new JobID(); - final JobVertexID vid = new JobVertexID(); - - JobVertex ajv = new JobVertex("TestVertex", vid); - ajv.setParallelism(3); - ajv.setInvokableClass(mock(AbstractInvokable.class).getClass()); - - ExecutionGraph graph = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jid, - "test job", - new Configuration(), - new ExecutionConfig(), - AkkaUtils.getDefaultTimeout(), - new NoRestartStrategy()); - graph.attachJobGraph(Arrays.asList(ajv)); - - setGraphStatus(graph, JobStatus.RUNNING); - - ExecutionJobVertex ejv = graph.getJobVertex(vid); - - // mock resources and mock taskmanager - for (ExecutionVertex ee : ejv.getTaskVertices()) { - SimpleSlot slot = getInstance( - new SimpleActorGateway( - TestingUtils.defaultExecutionContext()) - ).allocateSimpleSlot(jid); - ee.deployToSlot(slot); - } - - // finish all - for (ExecutionVertex ee : ejv.getTaskVertices()) { - ee.executionFinished(); - } - - assertTrue(ejv.isInFinalState()); - assertEquals(JobStatus.FINISHED, graph.getState()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } -} \ No newline at end of file +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph; + +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*; +import static org.junit.Assert.*; +import static org.mockito.Mockito.mock; + +import java.util.Arrays; + +import org.apache.flink.api.common.ExecutionConfigTest; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; +import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.junit.Test; + +public class ExecutionStateProgressTest { + + @Test + public void testAccumulatedStateFinished() { + try { + final JobID jid = new JobID(); + final JobVertexID vid = new JobVertexID(); + + JobVertex ajv = new JobVertex("TestVertex", vid); + ajv.setParallelism(3); + ajv.setInvokableClass(mock(AbstractInvokable.class).getClass()); + + ExecutionGraph graph = new ExecutionGraph( + TestingUtils.defaultExecutionContext(), + jid, + "test job", + new Configuration(), + ExecutionConfigTest.getSerializedConfig(), + AkkaUtils.getDefaultTimeout(), + new NoRestartStrategy()); + graph.attachJobGraph(Arrays.asList(ajv)); + + setGraphStatus(graph, JobStatus.RUNNING); + + ExecutionJobVertex ejv = graph.getJobVertex(vid); + + // mock resources and mock taskmanager + for (ExecutionVertex ee : ejv.getTaskVertices()) { + SimpleSlot slot = getInstance( + new SimpleActorGateway( + TestingUtils.defaultExecutionContext()) + ).allocateSimpleSlot(jid); + ee.deployToSlot(slot); + } + + // finish all + for (ExecutionVertex ee : ejv.getTaskVertices()) { + ee.executionFinished(); + } + + assertTrue(ejv.isInFinalState()); + assertEquals(JobStatus.FINISHED, graph.getState()); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java index a4c86e3386250..d7ce0bafa78e9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java @@ -25,7 +25,7 @@ import java.net.InetAddress; import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.io.StrictlyLocalAssignment; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.core.io.InputSplitSource; @@ -266,14 +266,14 @@ public void testMultipleInstancesPerHost() { vertex.setInvokableClass(DummyInvokable.class); vertex.setInputSplitSource(new TestInputSplitSource(splits)); - JobGraph jobGraph = new JobGraph("test job", new ExecutionConfig(), vertex); + JobGraph jobGraph = new JobGraph("test job", vertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), TIMEOUT, new NoRestartStrategy()); @@ -331,14 +331,14 @@ private static String[] runTests(int numHosts, int slotsPerHost, int parallelism vertex.setInvokableClass(DummyInvokable.class); vertex.setInputSplitSource(new TestInputSplitSource(splits)); - JobGraph jobGraph = new JobGraph("test job", new ExecutionConfig(), vertex); + JobGraph jobGraph = new JobGraph("test job", vertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jobGraph.getJobID(), jobGraph.getName(), jobGraph.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), TIMEOUT, new NoRestartStrategy()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java index cbeeded68121c..1b369db8359fe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java @@ -22,7 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -64,7 +64,7 @@ public void testNToN() { jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -105,8 +105,8 @@ public void test2NToN() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, - new ExecutionConfig(), + cfg, + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -148,8 +148,8 @@ public void test3NToN() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, - new ExecutionConfig(), + cfg, + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -192,8 +192,8 @@ public void testNTo2N() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, - new ExecutionConfig(), + cfg, + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -234,8 +234,8 @@ public void testNTo7N() { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, - new ExecutionConfig(), + cfg, + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -296,8 +296,8 @@ private void testLowToHigh(int lowDop, int highDop) { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, - new ExecutionConfig(), + cfg, + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { @@ -349,8 +349,8 @@ private void testHighToLow(int highDop, int lowDop) { TestingUtils.defaultExecutionContext(), jobId, jobName, - cfg, - new ExecutionConfig(), + cfg, + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index a28fb491d02bd..8bc474bec16ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -188,7 +188,7 @@ static class TestExecGraph extends ExecutionGraph { jobId, "test graph", EMPTY_CONFIG, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), TIMEOUT, new FixedDelayRestartStrategy(1, 0)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index d866b2f7ac826..c483f41f588d5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -26,7 +26,7 @@ import java.util.Collections; import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.DummyActorGateway; @@ -76,14 +76,14 @@ public void testScheduleWithConstraint1() { JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); jobVertex.setInvokableClass(DummyInvokable.class); jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); + JobGraph jg = new JobGraph("test job", jobVertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); @@ -149,14 +149,14 @@ public void testScheduleWithConstraint2() { JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); jobVertex.setInvokableClass(DummyInvokable.class); jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); + JobGraph jg = new JobGraph("test job", jobVertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); @@ -226,14 +226,14 @@ public void testScheduleWithConstraintAndSlotSharing() { jobVertex1.setSlotSharingGroup(sharingGroup); jobVertex2.setSlotSharingGroup(sharingGroup); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex1, jobVertex2); + JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); @@ -294,14 +294,14 @@ public void testScheduleWithUnfulfillableConstraint() { JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); jobVertex.setInvokableClass(DummyInvokable.class); jobVertex.setParallelism(1); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); + JobGraph jg = new JobGraph("test job", jobVertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(jobVertex)); @@ -360,7 +360,7 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { jobVertex1.setParallelism(1); jobVertex2.setParallelism(1); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex1, jobVertex2); + JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); SlotSharingGroup sharingGroup = new SlotSharingGroup(); jobVertex1.setSlotSharingGroup(sharingGroup); @@ -371,7 +371,7 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); @@ -404,14 +404,14 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { public void testArchivingClearsFields() { try { JobVertex vertex = new JobVertex("test vertex", new JobVertexID()); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), vertex); + JobGraph jg = new JobGraph("test job", vertex); ExecutionGraph eg = new ExecutionGraph( TestingUtils.defaultExecutionContext(), jg.getJobID(), jg.getName(), jg.getJobConfiguration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), timeout, new NoRestartStrategy()); eg.attachJobGraph(Collections.singletonList(vertex)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java index 511024902851c..7a23e26fd6103 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java @@ -24,7 +24,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; @@ -76,7 +76,7 @@ public void testAssignSlotSharingGroup() { new JobID(), "test job", new Configuration(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy()); eg.attachJobGraph(vertices); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index 317eed7edf47d..af8aa694c3c05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -89,8 +89,7 @@ public void testPartialConsumePipelinedResultReceiver() throws Exception { receiver.connectNewDataSetAsInput( sender, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); - final JobGraph jobGraph = new JobGraph( - "Partial Consume of Pipelined Result", new ExecutionConfig(), sender, receiver); + final JobGraph jobGraph = new JobGraph("Partial Consume of Pipelined Result", sender, receiver); final SlotSharingGroup slotSharingGroup = new SlotSharingGroup( sender.getID(), receiver.getID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java index 68b05b209a01c..74f1adf04223f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java @@ -22,7 +22,6 @@ import java.util.List; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; @@ -32,7 +31,7 @@ public class JobGraphTest { @Test public void testSerialization() { try { - JobGraph jg = new JobGraph("The graph", new ExecutionConfig()); + JobGraph jg = new JobGraph("The graph"); // add some configuration values { @@ -91,7 +90,7 @@ public void testTopologicalSort1() { intermediate2.connectNewDataSetAsInput(intermediate1, DistributionPattern.POINTWISE); intermediate1.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE); - JobGraph graph = new JobGraph("TestGraph", new ExecutionConfig(), + JobGraph graph = new JobGraph("TestGraph", source1, source2, intermediate1, intermediate2, target1, target2); List sorted = graph.getVerticesSortedTopologicallyFromSources(); @@ -136,7 +135,7 @@ public void testTopologicalSort2() { l13.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE); - JobGraph graph = new JobGraph("TestGraph", new ExecutionConfig(), + JobGraph graph = new JobGraph("TestGraph", source1, source2, root, l11, l13, l12, l2); List sorted = graph.getVerticesSortedTopologicallyFromSources(); @@ -183,7 +182,7 @@ public void testTopologicalSort3() { op2.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE); op3.connectNewDataSetAsInput(op2, DistributionPattern.POINTWISE); - JobGraph graph = new JobGraph("TestGraph", new ExecutionConfig(), source, op1, op2, op3); + JobGraph graph = new JobGraph("TestGraph", source, op1, op2, op3); List sorted = graph.getVerticesSortedTopologicallyFromSources(); assertEquals(4, sorted.size()); @@ -212,7 +211,7 @@ public void testTopoSortCyclicGraphNoSources() { v3.connectNewDataSetAsInput(v2, DistributionPattern.POINTWISE); v4.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE); - JobGraph jg = new JobGraph("Cyclic Graph", new ExecutionConfig(), v1, v2, v3, v4); + JobGraph jg = new JobGraph("Cyclic Graph", v1, v2, v3, v4); try { jg.getVerticesSortedTopologicallyFromSources(); fail("Failed to raise error on topologically sorting cyclic graph."); @@ -244,7 +243,7 @@ public void testTopoSortCyclicGraphIntermediateCycle() { v4.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE); target.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE); - JobGraph jg = new JobGraph("Cyclic Graph", new ExecutionConfig(), v1, v2, v3, v4, source, target); + JobGraph jg = new JobGraph("Cyclic Graph", v1, v2, v3, v4, source, target); try { jg.getVerticesSortedTopologicallyFromSources(); fail("Failed to raise error on topologically sorting cyclic graph."); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java index 612f64f9c9f25..d1d5f03b2ebc4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java @@ -67,7 +67,7 @@ public void testGeneratorWithoutAnyAttachements() { sink1.connectNewDataSetAsInput(join2, DistributionPattern.POINTWISE); sink2.connectNewDataSetAsInput(join1, DistributionPattern.ALL_TO_ALL); - JobGraph jg = new JobGraph("my job", new ExecutionConfig(), source1, source2, source3, + JobGraph jg = new JobGraph("my job", source1, source2, source3, intermediate1, intermediate2, join1, join2, sink1, sink2); String plan = JsonPlanGenerator.generatePlan(jg); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java index 3785fc701ef65..5c25003bb7e78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java @@ -126,7 +126,7 @@ protected void run() { sender.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block sender.createAndAddResultDataSet(rid, PIPELINED); - final JobGraph jobGraph = new JobGraph("Blocking test job", new ExecutionConfig(), sender); + final JobGraph jobGraph = new JobGraph("Blocking test job", sender); final JobID jid = jobGraph.getJobID(); final ActorGateway jobManagerGateway = cluster.getLeaderGateway( @@ -253,7 +253,7 @@ protected void run() { sender.setParallelism(2); sender.setInvokableClass(StoppableInvokable.class); - final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", new ExecutionConfig(), sender); + final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", sender); final JobID jid = jobGraph.getJobID(); final ActorGateway jobManagerGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION()); @@ -305,7 +305,7 @@ protected void run() { sender.setParallelism(1); sender.setInvokableClass(Tasks.BlockingNoOpInvokable.class); // just block - final JobGraph jobGraph = new JobGraph("Non-Stoppable batching test job", new ExecutionConfig(), sender); + final JobGraph jobGraph = new JobGraph("Non-Stoppable batching test job", sender); final JobID jid = jobGraph.getJobID(); final ActorGateway jobManagerGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java index e820ed68214be..959b9a75d53a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java @@ -109,7 +109,7 @@ public void testFailureWhenJarBlobsMissing() { // create a simple job graph JobVertex jobVertex = new JobVertex("Test Vertex"); jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); + JobGraph jg = new JobGraph("test job", jobVertex); // request the blob port from the job manager Future future = jmGateway.ask(JobManagerMessages.getRequestBlobManagerPort(), timeout); @@ -173,7 +173,7 @@ public void initializeOnMaster(ClassLoader loader) throws Exception { }; jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - JobGraph jg = new JobGraph("test job", new ExecutionConfig(), jobVertex); + JobGraph jg = new JobGraph("test job", jobVertex); // submit the job Future submitFuture = jmGateway.ask( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java index dfb0b91d11eec..561bda304f805 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java @@ -104,7 +104,7 @@ private JobGraph createTestJobGraph( DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - final JobGraph jobGraph = new JobGraph(jobName, new ExecutionConfig(), sender, receiver); + final JobGraph jobGraph = new JobGraph(jobName, sender, receiver); // We need to allow queued scheduling, because there are not enough slots available // to run all tasks at once. We queue tasks and then let them finish/consume the blocking diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java index ca2ecf515ec54..8ebb7f8da480c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java @@ -38,7 +38,7 @@ public void testNoOps() throws Exception { StandaloneSubmittedJobGraphStore jobGraphs = new StandaloneSubmittedJobGraphStore(); SubmittedJobGraph jobGraph = new SubmittedJobGraph( - new JobGraph("testNoOps", new ExecutionConfig()), + new JobGraph("testNoOps"), new JobInfo(ActorRef.noSender(), ListeningBehaviour.DETACHED, 0, Integer.MAX_VALUE)); assertEquals(0, jobGraphs.recoverJobGraphs().size()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java index 5e53596fab28f..c71bd355d01f8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java @@ -261,7 +261,7 @@ public void testUpdateJobGraphYouDidNotGetOrAdd() throws Exception { // --------------------------------------------------------------------------------------------- private SubmittedJobGraph createSubmittedJobGraph(JobID jobId, long start) { - final JobGraph jobGraph = new JobGraph(jobId, "Test JobGraph", new ExecutionConfig()); + final JobGraph jobGraph = new JobGraph(jobId, "Test JobGraph"); final JobVertex jobVertex = new JobVertex("Test JobVertex"); jobVertex.setParallelism(1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java index 07fc2c550e1ed..b03c38bc9334b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java @@ -113,7 +113,6 @@ public void testMixedPipelinedAndBlockingResults() throws Exception { final JobGraph jobGraph = new JobGraph( "Mixed pipelined and blocking result", - new ExecutionConfig(), sender, pipelinedReceiver, blockingReceiver); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java index a2cefb69aa867..ccd2156200274 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeJobRecoveryTest.java @@ -140,7 +140,10 @@ public JobGraph createBlockingJob(int parallelism) { ExecutionConfig executionConfig = new ExecutionConfig(); - return new JobGraph("Blocking test job", executionConfig, sender, receiver); + JobGraph jobGraph = new JobGraph("Blocking test job", sender, receiver); + jobGraph.setExecutionConfig(executionConfig); + + return jobGraph; } public static class TestActorGateway implements ActorGateway { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java index 6d938acf2c2ea..2eacdeec1b1e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java @@ -269,6 +269,6 @@ public JobGraph createBlockingJob(int parallelism) { sender.setSlotSharingGroup(slotSharingGroup); receiver.setSlotSharingGroup(slotSharingGroup); - return new JobGraph("Blocking test job", new ExecutionConfig(), sender, receiver); + return new JobGraph("Blocking test job", sender, receiver); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 9d33920b96ba4..1b463bcd38702 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.taskmanager; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; @@ -149,7 +149,8 @@ private static Task createTask() { TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), new JobVertexID(), new ExecutionAttemptID(), - new ExecutionConfig(), "Test Task", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), + "Test Task", 0, 1, 0, new Configuration(), new Configuration(), CheckpointsInOrderInvokable.class.getName(), Collections.emptyList(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java index f0e72d710cc34..a093233f2c175 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java @@ -97,7 +97,7 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { producer.setSlotSharingGroup(slot); consumer.setSlotSharingGroup(slot); - JobGraph jobGraph = new JobGraph(new ExecutionConfig(), producer, consumer); + JobGraph jobGraph = new JobGraph(producer, consumer); // Submit job and wait until running ActorGateway jobManager = flink.getLeaderGateway(deadline.timeLeft()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java index e7f4c5c4a2bee..09dd8170c262d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java @@ -74,7 +74,7 @@ public void testCancelUnion() throws Exception { flink.start(); // Setup - final JobGraph jobGraph = new JobGraph("Cancel Big Union", new ExecutionConfig()); + final JobGraph jobGraph = new JobGraph("Cancel Big Union"); JobVertex[] sources = new JobVertex[numberOfSources]; SlotSharingGroup group = new SlotSharingGroup(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 9e7abb6bb6bff..3ee9a842aa39e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -24,6 +24,7 @@ import akka.japi.Creator; import akka.testkit.JavaTestKit; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -66,6 +67,7 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.StoppableInvokable; import org.apache.flink.util.NetUtils; +import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -158,7 +160,7 @@ protected void run() { final JobID jid = new JobID(); final JobVertexID vid = new JobVertexID(); final ExecutionAttemptID eid = new ExecutionAttemptID(); - final ExecutionConfig executionConfig = new ExecutionConfig(); + final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, executionConfig, "TestTask", 2, 7, 0, new Configuration(), new Configuration(), @@ -262,14 +264,14 @@ public void testJobSubmissionAndCanceling() { final ExecutionAttemptID eid2 = new ExecutionAttemptID(); final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, - new ExecutionConfig(), "TestTask1", 1, 5, 0, + ExecutionConfigTest.getSerializedConfig(), "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, - new ExecutionConfig(), "TestTask2", 2, 7, 0, + ExecutionConfigTest.getSerializedConfig(), "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -395,13 +397,15 @@ public void testJobSubmissionAndStop() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, new ExecutionConfig(), + final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); + + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, executionConfig, "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), StoppableInvokable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, new ExecutionConfig(), + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, executionConfig, "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -522,14 +526,14 @@ public void testGateChannelEdgeMismatch() { final ExecutionAttemptID eid2 = new ExecutionAttemptID(); final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, - new ExecutionConfig(), "Sender", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, - new ExecutionConfig(), "Receiver", 2, 7, 0, + ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -623,13 +627,13 @@ public void testRunJobWithForwardChannel() { ); final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, - new ExecutionConfig(), "Sender", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, - new ExecutionConfig(), "Receiver", 2, 7, 0, + ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), Collections.singletonList(ircdd), @@ -764,13 +768,13 @@ public void testCancellingDependentAndStateUpdateFails() { ); final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, - new ExecutionConfig(), "Sender", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, - new ExecutionConfig(), "Receiver", 2, 7, 0, + ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.BlockingReceiver.class.getName(), Collections.emptyList(), Collections.singletonList(ircdd), @@ -909,7 +913,7 @@ public void testRemotePartitionNotFound() throws Exception { final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( jid, vid, eid, - new ExecutionConfig(), "Receiver", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), Collections.emptyList(), @@ -1003,7 +1007,7 @@ public void testLocalPartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, new ExecutionConfig(), "Receiver", 0, 1, 0, + jid, vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), Collections.emptyList(), @@ -1079,7 +1083,7 @@ public void testTriggerStackTraceSampleMessage() throws Exception { new JobID(), new JobVertexID(), new ExecutionAttemptID(), - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), "Task", 0, 1, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index aa37d4769487d..99e037d452b00 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.util.SerializedValue; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -62,7 +63,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { when(tddMock.getExecutionId()).thenReturn(mock(ExecutionAttemptID.class)); when(tddMock.getJobConfiguration()).thenReturn(mock(Configuration.class)); when(tddMock.getTaskConfiguration()).thenReturn(mock(Configuration.class)); - when(tddMock.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(tddMock.getSerializedExecutionConfig()).thenReturn(mock(SerializedValue.class)); when(tddMock.getInvokableClassName()).thenReturn("className"); task = new Task(tddMock, mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 034681e5c32c3..06f393f30c4a9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -20,7 +20,7 @@ import com.google.common.collect.Maps; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -628,7 +628,8 @@ private Task createTask(Class invokable, private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { return new TaskDeploymentDescriptor( new JobID(), new JobVertexID(), new ExecutionAttemptID(), - new ExecutionConfig(), "Test Task", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), + "Test Task", 0, 1, 0, new Configuration(), new Configuration(), invokable.getName(), Collections.emptyList(), diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala index ef3dae42203c9..1927c39f48cc1 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/TaskManagerLossFailsTasksTest.scala @@ -18,12 +18,12 @@ package org.apache.flink.runtime.executiongraph -import org.apache.flink.api.common.{ExecutionConfig, JobID} +import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest, JobID} import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.akka.AkkaUtils import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy -import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, JobVertex} +import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks import org.apache.flink.runtime.jobmanager.scheduler.Scheduler import org.apache.flink.runtime.testingUtils.TestingUtils @@ -50,14 +50,14 @@ class TaskManagerLossFailsTasksTest extends WordSpecLike with Matchers { sender.setInvokableClass(classOf[Tasks.NoOpInvokable]) sender.setParallelism(20) - val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender) + val jobGraph = new JobGraph("Pointwise job", sender) val eg = new ExecutionGraph( TestingUtils.defaultExecutionContext, new JobID(), "test job", new Configuration(), - new ExecutionConfig, + ExecutionConfigTest.getSerializedConfig, AkkaUtils.getDefaultTimeout, new NoRestartStrategy()) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala index f52d37e5ee854..12e2d638ecb98 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.scala @@ -68,7 +68,7 @@ class CoLocationConstraintITCase(_system: ActorSystem) receiver.setStrictlyCoLocatedWith(sender) - val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala index 894ba389ab366..2b5b29f48ef42 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala @@ -68,7 +68,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(2) vertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex) + val jobGraph = new JobGraph("Test Job", vertex) val cluster = TestingUtils.startTestingCluster(1) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -110,7 +110,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(num_tasks) vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex) + val jobGraph = new JobGraph("Test Job", vertex) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -145,7 +145,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(num_tasks) vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test job", new ExecutionConfig(), vertex) + val jobGraph = new JobGraph("Test job", vertex) jobGraph.setAllowQueuedScheduling(true) val cluster = TestingUtils.startTestingCluster(10) @@ -181,7 +181,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val cluster = TestingUtils.startTestingCluster(2 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -216,7 +216,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Bipartite Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Bipartite Job", sender, receiver) val cluster = TestingUtils.startTestingCluster(2 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -253,8 +253,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) receiver.connectNewDataSetAsInput(sender2, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Bipartite Job", new ExecutionConfig(), - sender1, receiver, sender2) + val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) val cluster = TestingUtils.startTestingCluster(6 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -299,8 +298,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) receiver.connectNewDataSetAsInput(sender2, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Bipartite Job", new ExecutionConfig(), - sender1, receiver, sender2) + val jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2) val cluster = TestingUtils.startTestingCluster(6 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -342,8 +340,7 @@ class JobManagerITCase(_system: ActorSystem) forwarder.connectNewDataSetAsInput(sender, DistributionPattern.ALL_TO_ALL) receiver.connectNewDataSetAsInput(forwarder, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Forwarding Job", new ExecutionConfig(), - sender, forwarder, receiver) + val jobGraph = new JobGraph("Forwarding Job", sender, forwarder, receiver) jobGraph.setScheduleMode(ScheduleMode.ALL) @@ -379,7 +376,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -427,7 +424,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -472,7 +469,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) val cluster = TestingUtils.startTestingCluster(2 * num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -512,7 +509,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -560,7 +557,7 @@ class JobManagerITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -603,8 +600,7 @@ class JobManagerITCase(_system: ActorSystem) source.setParallelism(num_tasks) sink.setParallelism(num_tasks) - val jobGraph = new JobGraph("SubtaskInFinalStateRaceCondition", - new ExecutionConfig(), source, sink) + val jobGraph = new JobGraph("SubtaskInFinalStateRaceCondition", source, sink) val cluster = TestingUtils.startTestingCluster(2*num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -630,12 +626,12 @@ class JobManagerITCase(_system: ActorSystem) val vertex = new JobVertex("Test Vertex") vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph1 = new JobGraph("Test Job", new ExecutionConfig(), vertex) + val jobGraph1 = new JobGraph("Test Job", vertex) val slowVertex = new WaitingOnFinalizeJobVertex("Long running Vertex", 2000) slowVertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph2 = new JobGraph("Long running Job", new ExecutionConfig(), slowVertex) + val jobGraph2 = new JobGraph("Long running Job", slowVertex) val cluster = TestingUtils.startTestingCluster(1) val jm = cluster.getLeaderGateway(1 seconds) @@ -684,7 +680,7 @@ class JobManagerITCase(_system: ActorSystem) vertex.setParallelism(1) vertex.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph = new JobGraph("Test Job", new ExecutionConfig(), vertex) + val jobGraph = new JobGraph("Test Job", vertex) val cluster = TestingUtils.startTestingCluster(1) val jm = cluster.getLeaderGateway(1 seconds) @@ -782,7 +778,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) + val jobGraph = new JobGraph(jobVertex) // Submit job w/o checkpointing configured jobManager.tell(SubmitJob(jobGraph, ListeningBehaviour.DETACHED), testActor) @@ -815,7 +811,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) + val jobGraph = new JobGraph(jobVertex) jobGraph.setSnapshotSettings(new JobSnapshottingSettings( java.util.Collections.emptyList(), java.util.Collections.emptyList(), @@ -868,7 +864,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) + val jobGraph = new JobGraph(jobVertex) jobGraph.setSnapshotSettings(new JobSnapshottingSettings( java.util.Collections.emptyList(), java.util.Collections.emptyList(), @@ -926,7 +922,7 @@ class JobManagerITCase(_system: ActorSystem) val jobVertex = new JobVertex("Blocking vertex") jobVertex.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph(new ExecutionConfig(), jobVertex) + val jobGraph = new JobGraph(jobVertex) jobGraph.setSnapshotSettings(new JobSnapshottingSettings( java.util.Collections.emptyList(), java.util.Collections.emptyList(), diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala index ea42cd14c7aa5..b96369fd36f67 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/RecoveryITCase.scala @@ -86,7 +86,8 @@ class RecoveryITCase(_system: ActorSystem) val executionConfig = new ExecutionConfig() executionConfig.setNumberOfExecutionRetries(1); - val jobGraph = new JobGraph("Pointwise job", executionConfig, sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) + jobGraph.setExecutionConfig(executionConfig) val cluster = createTestClusterWithHeartbeatTimeout(2 * NUM_TASKS, 1, "2 s") cluster.start() @@ -133,7 +134,8 @@ class RecoveryITCase(_system: ActorSystem) val executionConfig = new ExecutionConfig() executionConfig.setNumberOfExecutionRetries(1); - val jobGraph = new JobGraph("Pointwise job", executionConfig, sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) + jobGraph.setExecutionConfig(executionConfig) val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 1, "2 s") cluster.start() @@ -180,7 +182,8 @@ class RecoveryITCase(_system: ActorSystem) val executionConfig = new ExecutionConfig() executionConfig.setNumberOfExecutionRetries(1); - val jobGraph = new JobGraph("Pointwise job", executionConfig, sender, receiver) + val jobGraph = new JobGraph("Pointwise job", sender, receiver) + jobGraph.setExecutionConfig(executionConfig) val cluster = createTestClusterWithHeartbeatTimeout(NUM_TASKS, 2, "2 s") cluster.start() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala index 4d320eabdadcb..f986e7374d7a1 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/SlotSharingITCase.scala @@ -66,7 +66,7 @@ class SlotSharingITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) @@ -110,8 +110,7 @@ class SlotSharingITCase(_system: ActorSystem) receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE) receiver.connectNewDataSetAsInput(sender2, DistributionPattern.ALL_TO_ALL) - val jobGraph = new JobGraph("Bipartite job", new ExecutionConfig(), - sender1, sender2, receiver) + val jobGraph = new JobGraph("Bipartite job", sender1, sender2, receiver) val cluster = TestingUtils.startTestingCluster(num_tasks) val jmGateway = cluster.getLeaderGateway(1 seconds) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala index 100fe662ada08..d0136f05dd9c8 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/TaskManagerFailsWithSlotSharingITCase.scala @@ -67,7 +67,7 @@ class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val jobID = jobGraph.getJobID val cluster = TestingUtils.startTestingCluster(num_tasks/2, 2) @@ -116,7 +116,7 @@ class TaskManagerFailsWithSlotSharingITCase(_system: ActorSystem) sender.setSlotSharingGroup(sharingGroup) receiver.setSlotSharingGroup(sharingGroup) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val jobID = jobGraph.getJobID val cluster = TestingUtils.startTestingCluster(num_tasks/2, 2) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index c3b515e888fec..9adabae33ec04 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -55,7 +55,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.nio.charset.Charset; import java.util.ArrayDeque; import java.util.ArrayList; @@ -110,7 +109,8 @@ private void init() { } public JobGraph createJobGraph() { - jobGraph = new JobGraph(streamGraph.getJobName(), streamGraph.getExecutionConfig()); + + jobGraph = new JobGraph(streamGraph.getJobName()); // make sure that all vertices start immediately jobGraph.setScheduleMode(ScheduleMode.ALL); @@ -126,15 +126,11 @@ public JobGraph createJobGraph() { setPhysicalEdges(); setSlotSharing(); - + configureCheckpointing(); - try { - // make sure that we can send the ExecutionConfig without user code object problems - jobGraph.getExecutionConfig().serializeUserCode(); - } catch (IOException e) { - throw new IllegalStateException("Could not serialize ExecutionConfig.", e); - } + // set the ExecutionConfig last when it has been finalized + jobGraph.setExecutionConfig(streamGraph.getExecutionConfig()); return jobGraph; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java index 05175764f894a..f768ace808147 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/RestartStrategyTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -42,7 +43,8 @@ public void testAutomaticRestartingWhenCheckpointing() { StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); - RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getExecutionConfig().getRestartStrategy(); + RestartStrategies.RestartStrategyConfiguration restartStrategy = + ExecutionConfigTest.deserializeConfig(jobGraph.getSerializedExecutionConfig()).getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.FixedDelayRestartStrategyConfiguration); @@ -64,7 +66,8 @@ public void testNoRestartingWhenCheckpointingAndExplicitExecutionRetriesZero() { StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); - RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getExecutionConfig().getRestartStrategy(); + RestartStrategies.RestartStrategyConfiguration restartStrategy = + ExecutionConfigTest.deserializeConfig(jobGraph.getSerializedExecutionConfig()).getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.NoRestartStrategyConfiguration); @@ -86,7 +89,8 @@ public void testFixedRestartingWhenCheckpointingAndExplicitExecutionRetriesNonZe StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); - RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getExecutionConfig().getRestartStrategy(); + RestartStrategies.RestartStrategyConfiguration restartStrategy = + ExecutionConfigTest.deserializeConfig(jobGraph.getSerializedExecutionConfig()).getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.FixedDelayRestartStrategyConfiguration); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java index e9aec4887dd92..0de43254af0c1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java @@ -21,6 +21,7 @@ import java.util.Random; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -29,6 +30,7 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.SerializedValue; import org.junit.Test; import static org.junit.Assert.*; @@ -80,17 +82,19 @@ public void testExecutionConfigSerialization() throws IOException, ClassNotFound final String EXEC_CONFIG_KEY = "runtime.config"; - InstantiationUtil.writeObjectToConfig(jobGraph.getExecutionConfig(), + InstantiationUtil.writeObjectToConfig(jobGraph.getSerializedExecutionConfig(), jobGraph.getJobConfiguration(), EXEC_CONFIG_KEY); - ExecutionConfig executionConfig = InstantiationUtil.readObjectFromConfig( + SerializedValue serializedExecutionConfig = InstantiationUtil.readObjectFromConfig( jobGraph.getJobConfiguration(), EXEC_CONFIG_KEY, Thread.currentThread().getContextClassLoader()); - - assertNotNull(executionConfig); - + + assertNotNull(serializedExecutionConfig); + + ExecutionConfig executionConfig = ExecutionConfigTest.deserializeConfig(serializedExecutionConfig); + assertEquals(closureCleanerEnabled, executionConfig.isClosureCleanerEnabled()); assertEquals(forceAvroEnabled, executionConfig.isForceAvroEnabled()); assertEquals(forceKryoEnabled, executionConfig.isForceKryoEnabled()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java index 732d3e5404f80..8484e90787b91 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java @@ -17,7 +17,7 @@ package org.apache.flink.streaming.runtime.partitioner; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple; @@ -135,7 +135,7 @@ public void flatMap(String value, jobId, jobName, cfg, - new ExecutionConfig(), + ExecutionConfigTest.getSerializedConfig(), AkkaUtils.getDefaultTimeout(), new NoRestartStrategy(), new ArrayList(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 7f4492a74dbde..ed8bf019f7e02 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -20,7 +20,7 @@ import akka.actor.ActorRef; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; @@ -135,7 +135,8 @@ private Task createTask(Class invokable, StreamConf TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), new JobVertexID(), new ExecutionAttemptID(), - new ExecutionConfig(), "Test Task", 0, 1, 0, + ExecutionConfigTest.getSerializedConfig(), + "Test Task", 0, 1, 0, new Configuration(), taskConfig.getConfiguration(), invokable.getName(), diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java index f6c22d439f958..28c2e5837fc95 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.failingPrograms; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -65,7 +64,7 @@ public static void setup() { final JobVertex jobVertex = new JobVertex("Working job vertex."); jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - workingJobGraph = new JobGraph("Working testing job", new ExecutionConfig(), jobVertex); + workingJobGraph = new JobGraph("Working testing job", jobVertex); } catch (Exception e) { e.printStackTrace(); @@ -116,7 +115,7 @@ public void testExceptionInInitializeOnMaster() { final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex"); failingJobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - final JobGraph failingJobGraph = new JobGraph("Failing testing job", new ExecutionConfig(), failingJobVertex); + final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex); try { submitJob(failingJobGraph); @@ -141,7 +140,7 @@ public void testExceptionInInitializeOnMaster() { @Test public void testSubmitEmptyJobGraph() { try { - final JobGraph jobGraph = new JobGraph("Testing job", new ExecutionConfig()); + final JobGraph jobGraph = new JobGraph("Testing job"); try { submitJob(jobGraph); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java index f4d88a8cf0914..b9284dc3d7e7f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHACheckpointRecoveryITCase.java @@ -21,7 +21,6 @@ import akka.actor.ActorRef; import akka.actor.ActorSystem; import org.apache.commons.io.FileUtils; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -368,7 +367,7 @@ public void testCheckpointRecoveryFailure() throws Exception { // BLocking JobGraph JobVertex blockingVertex = new JobVertex("Blocking vertex"); blockingVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class); - JobGraph jobGraph = new JobGraph(new ExecutionConfig(), blockingVertex); + JobGraph jobGraph = new JobGraph(blockingVertex); // Submit the job in detached mode leader.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java index 2418853635aca..b4ffbd4f22412 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAJobGraphRecoveryITCase.java @@ -440,7 +440,7 @@ public void awaitJobResult(long timeout) throws InterruptedException { * Creates a simple blocking JobGraph. */ private static JobGraph createBlockingJobGraph() { - JobGraph jobGraph = new JobGraph("Blocking program", new ExecutionConfig()); + JobGraph jobGraph = new JobGraph("Blocking program"); JobVertex jobVertex = new JobVertex("Blocking Vertex"); jobVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class); 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 a87057845b7da..06df46fede9de 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 @@ -97,7 +97,7 @@ protected JobGraph getJobGraph() throws Exception { private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender, boolean isSlowReceiver, int numSubtasks) { - JobGraph jobGraph = new JobGraph("Speed Test", new ExecutionConfig()); + JobGraph jobGraph = new JobGraph("Speed Test"); SlotSharingGroup sharingGroup = new SlotSharingGroup(); JobVertex producer = new JobVertex("Speed Test Producer"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 09f9cacdc57a8..45ee839bc1b97 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -22,7 +22,6 @@ import akka.actor.Kill; import akka.actor.PoisonPill; import org.apache.commons.io.FileUtils; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -168,7 +167,7 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { sender.setSlotSharingGroup(slotSharingGroup); receiver.setSlotSharingGroup(slotSharingGroup); - final JobGraph graph = new JobGraph("Blocking test job", new ExecutionConfig(), sender, receiver); + final JobGraph graph = new JobGraph("Blocking test job", sender, receiver); final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration); diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java index e7b37d7bc52e7..9cc90a15e4c17 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java @@ -24,7 +24,6 @@ import org.apache.commons.io.FileUtils; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -213,7 +212,7 @@ public void testStop() throws Exception { sender.setParallelism(2); sender.setInvokableClass(StoppableInvokable.class); - final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", new ExecutionConfig(), sender); + final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", sender); final JobID jid = jobGraph.getJobID(); cluster.submitJobDetached(jobGraph); @@ -251,7 +250,7 @@ public void testStopYarn() throws Exception { sender.setParallelism(2); sender.setInvokableClass(StoppableInvokable.class); - final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", new ExecutionConfig(), sender); + final JobGraph jobGraph = new JobGraph("Stoppable streaming test job", sender); final JobID jid = jobGraph.getJobID(); cluster.submitJobDetached(jobGraph); diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala index 2265b3b58ab44..ac661f3e727d2 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala @@ -21,7 +21,7 @@ package org.apache.flink.api.scala.runtime.jobmanager import akka.actor.{ActorSystem, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} import org.apache.flink.configuration.{ConfigConstants, Configuration} -import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest} import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, NoOpInvokable} @@ -94,12 +94,12 @@ class JobManagerFailsITCase(_system: ActorSystem) val sender = new JobVertex("BlockingSender") sender.setParallelism(num_slots) sender.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph("Blocking Testjob", new ExecutionConfig(), sender) + val jobGraph = new JobGraph("Blocking Testjob", sender) val noOp = new JobVertex("NoOpInvokable") noOp.setParallelism(num_slots) noOp.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph2 = new JobGraph("NoOp Testjob", new ExecutionConfig(), noOp) + val jobGraph2 = new JobGraph("NoOp Testjob", noOp) val cluster = startDeathwatchCluster(num_slots / 2, 2) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala index 9aa1e944682a4..78bc0ee27c109 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerLeaderSessionIDITSuite.scala @@ -65,7 +65,7 @@ class JobManagerLeaderSessionIDITSuite(_system: ActorSystem) val sender = new JobVertex("BlockingSender"); sender.setParallelism(numSlots) sender.setInvokableClass(classOf[BlockingUntilSignalNoOpInvokable]) - val jobGraph = new JobGraph("TestJob", new ExecutionConfig(), sender) + val jobGraph = new JobGraph("TestJob", sender) val oldSessionID = UUID.randomUUID() diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala index f1e115dd861e7..258f6dfceae04 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala @@ -20,21 +20,19 @@ package org.apache.flink.api.scala.runtime.taskmanager import akka.actor.{ActorSystem, Kill, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} -import org.apache.flink.api.common.ExecutionConfig - +import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest} import org.apache.flink.configuration.ConfigConstants import org.apache.flink.configuration.Configuration -import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils} +import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} import org.apache.flink.runtime.client.JobExecutionException -import org.apache.flink.runtime.jobgraph.{JobVertex, DistributionPattern, JobGraph} -import org.apache.flink.runtime.jobmanager.Tasks.{NoOpInvokable, BlockingNoOpInvokable, BlockingReceiver, Sender} +import org.apache.flink.runtime.jobgraph.{DistributionPattern, JobGraph, JobVertex} +import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, BlockingReceiver, NoOpInvokable, Sender} import org.apache.flink.runtime.messages.JobManagerMessages._ -import org.apache.flink.runtime.messages.TaskManagerMessages.{RegisteredAtJobManager, NotifyWhenRegisteredAtJobManager} +import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager} import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} import org.apache.flink.test.util.ForkableFlinkMiniCluster - import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} @@ -99,7 +97,7 @@ class TaskManagerFailsITCase(_system: ActorSystem) receiver.setParallelism(num_tasks) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val jobID = jobGraph.getJobID val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2) @@ -151,7 +149,7 @@ class TaskManagerFailsITCase(_system: ActorSystem) receiver.setParallelism(num_tasks) receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE) - val jobGraph = new JobGraph("Pointwise Job", new ExecutionConfig(), sender, receiver) + val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val jobID = jobGraph.getJobID val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2) @@ -190,12 +188,12 @@ class TaskManagerFailsITCase(_system: ActorSystem) val sender = new JobVertex("BlockingSender") sender.setParallelism(num_slots) sender.setInvokableClass(classOf[BlockingNoOpInvokable]) - val jobGraph = new JobGraph("Blocking Testjob", new ExecutionConfig(), sender) + val jobGraph = new JobGraph("Blocking Testjob", sender) val noOp = new JobVertex("NoOpInvokable") noOp.setParallelism(num_slots) noOp.setInvokableClass(classOf[NoOpInvokable]) - val jobGraph2 = new JobGraph("NoOp Testjob", new ExecutionConfig(), noOp) + val jobGraph2 = new JobGraph("NoOp Testjob", noOp) val cluster = createDeathwatchCluster(num_slots/2, 2) From 689317e13fa4bfb196c68436381616605d25d2d7 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 13 May 2016 15:42:57 +0200 Subject: [PATCH 18/70] [FLINK-3856] adapt test assertion to type stack changes Addition to bbd02d24bc7547e2c9384d713b20f86682cac08c. The java.lang.Date type shouldn't be an automatically Kryo registered anymore. --- .../apache/flink/test/javaApiOperators/GroupReduceITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java index 075e60c08443c..6f93722b39827 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java @@ -1033,7 +1033,7 @@ public void testGroupByGenericType() throws Exception { // check if automatic type registration with Kryo worked Assert.assertTrue(ec.getRegisteredKryoTypes().contains(BigInt.class)); - Assert.assertTrue(ec.getRegisteredKryoTypes().contains(java.sql.Date.class)); + Assert.assertFalse(ec.getRegisteredKryoTypes().contains(java.sql.Date.class)); String expected = null; From 60f00efb980eb72ce74b9b5114b0be7a4d4255fe Mon Sep 17 00:00:00 2001 From: "Ignacio N. Lucero Ascencio" Date: Fri, 13 May 2016 23:41:25 +0200 Subject: [PATCH 19/70] [FLINK-3912] [docs] Fix errors in Batch Scala API Documentation, Join section This closes #1991 --- docs/apis/batch/dataset_transformations.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/apis/batch/dataset_transformations.md b/docs/apis/batch/dataset_transformations.md index be9691c114f0f..2d7281717e1ed 100644 --- a/docs/apis/batch/dataset_transformations.md +++ b/docs/apis/batch/dataset_transformations.md @@ -644,8 +644,8 @@ In contrast to a reduce function, a group-reduce function is not implicitly combinable. In order to make a group-reduce function combinable it must implement the `GroupCombineFunction` interface. -**Important**: The generic input and output types of -the `GroupCombineFunction` interface must be equal to the generic input type +**Important**: The generic input and output types of +the `GroupCombineFunction` interface must be equal to the generic input type of the `GroupReduceFunction` as shown in the following example:
@@ -655,7 +655,7 @@ of the `GroupReduceFunction` as shown in the following example: // Combinable GroupReduceFunction that computes a sum. public class MyCombinableGroupReducer implements GroupReduceFunction, String>, - GroupCombineFunction, Tuple2> + GroupCombineFunction, Tuple2> { @Override public void reduce(Iterable> in, @@ -683,7 +683,7 @@ public class MyCombinableGroupReducer implements sum += curr.f1; } // emit tuple with key and sum - out.collect(new Tuple2<>(key, sum)); + out.collect(new Tuple2<>(key, sum)); } } ~~~ @@ -1379,8 +1379,8 @@ val ratings: DataSet[Ratings] = // [...] val weights: DataSet[(String, Double)] = // [...] val weightedRatings = ratings.join(weights).where("category").equalTo(0) { - (rating, weight, out: Collector[(String, Double)] => - if (weight._2 > 0.1) out.collect(left.name, left.points * right._2) + (rating, weight, out: Collector[(String, Double)]) => + if (weight._2 > 0.1) out.collect(rating.name, rating.points * weight._2) } ~~~ From ea342606130484ae3f9144ed14e5bf43a5f2eb99 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Mon, 9 May 2016 14:42:56 -0400 Subject: [PATCH 20/70] [FLINK-3768] [gelly] Local Clustering Coefficient The local clustering coefficient measures the connectedness of each vertex's neighborhood. Scores range from 0.0 (no edges between neighbors) to 1.0 (neighborhood is a clique). This closes #1896 --- docs/apis/batch/libs/gelly.md | 18 ++ .../examples/LocalClusteringCoefficient.java | 135 ++++++++ .../flink/graph/examples/TriangleListing.java | 125 ++++++++ .../LocalClusteringCoefficient.java | 252 +++++++++++++++ .../undirected/TriangleListing.java | 302 ++++++++++++++++++ .../apache/flink/graph/utils/Murmur3_32.java | 107 +++++++ .../LocalClusteringCoefficientTest.java | 81 +++++ .../undirected/TriangleListingTest.java | 78 +++++ 8 files changed, 1098 insertions(+) create mode 100644 flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/LocalClusteringCoefficient.java create mode 100644 flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/TriangleListing.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Murmur3_32.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java diff --git a/docs/apis/batch/libs/gelly.md b/docs/apis/batch/libs/gelly.md index c68001bcc9ffd..dd0b4c185cc9e 100644 --- a/docs/apis/batch/libs/gelly.md +++ b/docs/apis/batch/libs/gelly.md @@ -1831,6 +1831,7 @@ Gelly has a growing collection of graph algorithms for easily analyzing large-sc * [GSA Triangle Count](#gsa-triangle-count) * [Triangle Enumerator](#triangle-enumerator) * [Summarization](#summarization) +* [Local Clustering Coefficient](#local-clustering-coefficient) Gelly's library methods can be used by simply calling the `run()` method on the input graph: @@ -2050,6 +2051,23 @@ The algorithm takes a directed, vertex (and possibly edge) attributed graph as i vertex represents a group of vertices and each edge represents a group of edges from the input graph. Furthermore, each vertex and edge in the output graph stores the common group value and the number of represented elements. +### Local Clustering Coefficient + +#### Overview +The local clustering coefficient measures the connectedness of each vertex's neighborhood. Scores range from 0.0 (no +edges between neighbors) to 1.0 (neighborhood is a clique). + +#### Details +An edge between a vertex's neighbors is a triangle. Counting edges between neighbors is equivalent to counting the +number of triangles which include the vertex. The clustering coefficient score is the number of edges between neighbors +divided by the number of potential edges between neighbors. + +See the [Triangle Enumeration](#triangle-enumeration) library method for a detailed explanation of triangle enumeration. + +#### Usage +The algorithm takes a simple, undirected graph as input and outputs a `DataSet` of tuples containing the vertex ID, +vertex degree, and number of triangles containing the vertex. The vertex ID must be `Comparable` and `Copyable`. + {% top %} Graph Algorithms diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/LocalClusteringCoefficient.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/LocalClusteringCoefficient.java new file mode 100644 index 0000000000000..2465da8128306 --- /dev/null +++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/LocalClusteringCoefficient.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.examples; + +import org.apache.commons.math3.random.JDKRandomGenerator; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.CsvOutputFormat; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.asm.translate.LongValueToIntValue; +import org.apache.flink.graph.asm.translate.TranslateGraphIds; +import org.apache.flink.graph.generator.RMatGraph; +import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory; +import org.apache.flink.graph.generator.random.RandomGenerableFactory; +import org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; + +import java.text.NumberFormat; + +/** + * Driver for the library implementation of Local Clustering Coefficient. + * + * This example generates an undirected RMat graph with the given scale and + * edge factor then calculates the local clustering coefficient for each vertex. + * + * @see org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient + */ +public class LocalClusteringCoefficient { + + public static final int DEFAULT_SCALE = 10; + + public static final int DEFAULT_EDGE_FACTOR = 16; + + public static final boolean DEFAULT_CLIP_AND_FLIP = true; + + public static void main(String[] args) throws Exception { + // Set up the execution environment + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().enableObjectReuse(); + + ParameterTool parameters = ParameterTool.fromArgs(args); + + // Generate RMat graph + int scale = parameters.getInt("scale", DEFAULT_SCALE); + int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR); + + RandomGenerableFactory rnd = new JDKRandomGeneratorFactory(); + + long vertexCount = 1 << scale; + long edgeCount = vertexCount * edgeFactor; + + boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP); + + Graph graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount) + .setSimpleGraph(true, clipAndFlip) + .generate(); + + DataSet cc; + + if (scale > 32) { + cc = graph + .run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient()); + } else { + cc = graph + .run(new TranslateGraphIds(new LongValueToIntValue())) + .run(new org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient()); + } + + switch (parameters.get("output", "")) { + case "print": + for (Object e: cc.collect()) { + Result result = (Result)e; + System.out.println(result.toVerboseString()); + } + break; + + case "hash": + System.out.println(DataSetUtils.checksumHashCode(cc)); + break; + + case "csv": + String filename = parameters.get("filename"); + + String row_delimiter = parameters.get("row_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER); + String field_delimiter = parameters.get("field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER); + + cc.writeAsCsv(filename, row_delimiter, field_delimiter); + + env.execute(); + break; + default: + System.out.println("The local clustering coefficient measures the connectedness of each vertex's"); + System.out.println("neighborhood. Scores range from 0.0 (no edges between neighbors) to 1.0"); + System.out.println("(neighborhood is a clique)"); + System.out.println(""); + System.out.println("This algorithm returns tuples containing the vertex ID, the degree of"); + System.out.println("the vertex, the number of edges between vertex neighbors, and the local"); + System.out.println("clustering coefficient."); + System.out.println(""); + System.out.println("usage:"); + System.out.println(" LocalClusteringCoefficient [--scale SCALE] [--edge_factor EDGE_FACTOR] --output print"); + System.out.println(" LocalClusteringCoefficient [--scale SCALE] [--edge_factor EDGE_FACTOR] --output hash"); + System.out.println(" LocalClusteringCoefficient [--scale SCALE] [--edge_factor EDGE_FACTOR] --output csv" + + " --filename FILENAME [--row_delimiter ROW_DELIMITER] [--field_delimiter FIELD_DELIMITER]"); + + return; + } + + JobExecutionResult result = env.getLastJobExecutionResult(); + + NumberFormat nf = NumberFormat.getInstance(); + System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms"); + } +} diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/TriangleListing.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/TriangleListing.java new file mode 100644 index 0000000000000..f5f232d888020 --- /dev/null +++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/TriangleListing.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.examples; + +import org.apache.commons.math3.random.JDKRandomGenerator; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.CsvOutputFormat; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.asm.translate.LongValueToIntValue; +import org.apache.flink.graph.asm.translate.TranslateGraphIds; +import org.apache.flink.graph.generator.RMatGraph; +import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory; +import org.apache.flink.graph.generator.random.RandomGenerableFactory; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; + +import java.text.NumberFormat; + +/** + * Driver for the library implementation of Triangle Listing. + * + * This example generates an undirected RMat graph with the given scale + * and edge factor then lists all triangles. + * + * @see org.apache.flink.graph.library.clustering.undirected.TriangleListing + */ +public class TriangleListing { + + public static final int DEFAULT_SCALE = 10; + + public static final int DEFAULT_EDGE_FACTOR = 16; + + public static final boolean DEFAULT_CLIP_AND_FLIP = true; + + public static void main(String[] args) throws Exception { + // Set up the execution environment + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().enableObjectReuse(); + + ParameterTool parameters = ParameterTool.fromArgs(args); + + // Generate RMat graph + int scale = parameters.getInt("scale", DEFAULT_SCALE); + int edgeFactor = parameters.getInt("edge_factor", DEFAULT_EDGE_FACTOR); + + RandomGenerableFactory rnd = new JDKRandomGeneratorFactory(); + + long vertexCount = 1 << scale; + long edgeCount = vertexCount * edgeFactor; + + boolean clipAndFlip = parameters.getBoolean("clip_and_flip", DEFAULT_CLIP_AND_FLIP); + + Graph graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount) + .setSimpleGraph(true, clipAndFlip) + .generate(); + + DataSet tl; + + if (scale > 32) { + tl = graph + .run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing()); + } else { + tl = graph + .run(new TranslateGraphIds(new LongValueToIntValue())) + .run(new org.apache.flink.graph.library.clustering.undirected.TriangleListing()); + } + + switch (parameters.get("output", "")) { + case "print": + tl.print(); + break; + + case "hash": + System.out.println(DataSetUtils.checksumHashCode(tl)); + break; + + case "csv": + String filename = parameters.get("filename"); + + String row_delimiter = parameters.get("row_delimiter", CsvOutputFormat.DEFAULT_LINE_DELIMITER); + String field_delimiter = parameters.get("field_delimiter", CsvOutputFormat.DEFAULT_FIELD_DELIMITER); + + tl.writeAsCsv(filename, row_delimiter, field_delimiter); + + env.execute(); + break; + default: + System.out.println("Lists all distinct triangles in the generated RMat graph."); + System.out.println(""); + System.out.println("usage:"); + System.out.println(" TriangleListing [--scale SCALE] [--edge_factor EDGE_FACTOR] --output print"); + System.out.println(" TriangleListing [--scale SCALE] [--edge_factor EDGE_FACTOR] --output hash"); + System.out.println(" TriangleListing [--scale SCALE] [--edge_factor EDGE_FACTOR] --output csv" + + " --filename FILENAME [--row_delimiter ROW_DELIMITER] [--field_delimiter FIELD_DELIMITER]"); + + return; + } + + JobExecutionResult result = env.getLastJobExecutionResult(); + + NumberFormat nf = NumberFormat.getInstance(); + System.out.println("Execution runtime: " + nf.format(result.getNetRuntime()) + " ms"); + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java new file mode 100644 index 0000000000000..0a562d5ac2350 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficient.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.library.clustering.undirected; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.functions.FunctionAnnotation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.asm.degree.annotate.undirected.VertexDegree; +import org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result; +import org.apache.flink.graph.utils.Murmur3_32; +import org.apache.flink.types.CopyableValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.util.Collector; + +/** + * The local clustering coefficient measures the connectedness of each vertex's + * neighborhood. Scores range from 0.0 (no edges between neighbors) to 1.0 + * (neighborhood is a clique). + *
+ * An edge between a vertex's neighbors is a triangle. Counting edges between + * neighbors is equivalent to counting the number of triangles which include + * the vertex. + *
+ * The input graph must be a simple, undirected graph containing no duplicate + * edges or self-loops. + * + * @param graph ID type + * @param vertex value type + * @param edge value type + */ +public class LocalClusteringCoefficient & CopyableValue, VV, EV> +implements GraphAlgorithm>> { + + // Optional configuration + private int littleParallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * Override the parallelism of operators processing small amounts of data. + * + * @param littleParallelism operator parallelism + * @return this + */ + public LocalClusteringCoefficient setLittleParallelism(int littleParallelism) { + this.littleParallelism = littleParallelism; + + return this; + } + + /* + * Implementation notes: + * + * The requirement that "K extends CopyableValue" can be removed when + * removed from TriangleListing. + * + * CountVertices can be replaced by ".sum(1)" when Flink aggregators use + * code generation. + */ + + @Override + public DataSet> run(Graph input) + throws Exception { + // u, v, w + DataSet> triangles = input + .run(new TriangleListing() + .setSortTriangleVertices(false) + .setLittleParallelism(littleParallelism)); + + // u, 1 + DataSet> triangleVertices = triangles + .flatMap(new SplitTriangles()) + .name("Split triangle vertices"); + + // u, triangle count + DataSet> vertexTriangleCount = triangleVertices + .groupBy(0) + .reduce(new CountVertices()) + .name("Count triangles"); + + // u, deg(u) + DataSet> vertexDegree = input + .run(new VertexDegree() + .setParallelism(littleParallelism) + .setIncludeZeroDegreeVertices(true)); + + // u, deg(u), triangle count + return vertexDegree + .leftOuterJoin(vertexTriangleCount) + .where(0) + .equalTo(0) + .with(new JoinVertexDegreeWithTriangleCount()) + .setParallelism(littleParallelism) + .name("Clustering coefficient"); + } + + /** + * Emits the three vertex IDs comprising each triangle along with an initial count. + * + * @param ID type + */ + private static class SplitTriangles + implements FlatMapFunction, Tuple2> { + private Tuple2 output = new Tuple2<>(null, new LongValue(1)); + + @Override + public void flatMap(Tuple3 value, Collector> out) + throws Exception { + output.f0 = value.f0; + out.collect(output); + + output.f0 = value.f1; + out.collect(output); + + output.f0 = value.f2; + out.collect(output); + } + } + + /** + * Combines the count of each vertex ID. + * + * @param ID type + */ + @FunctionAnnotation.ForwardedFields("0") + private static class CountVertices + implements ReduceFunction> { + @Override + public Tuple2 reduce(Tuple2 left, Tuple2 right) + throws Exception { + left.f1.setValue(left.f1.getValue() + right.f1.getValue()); + return left; + } + } + + /** + * Joins the vertex and degree with the vertex's triangle count. + * + * @param ID type + */ + @FunctionAnnotation.ForwardedFieldsFirst("0; 1->1.0") + @FunctionAnnotation.ForwardedFieldsSecond("0") + private static class JoinVertexDegreeWithTriangleCount + implements JoinFunction, Tuple2, Result> { + private LongValue zero = new LongValue(0); + + private Result output = new Result<>(); + + @Override + public Result join(Vertex vertexAndDegree, Tuple2 vertexAndTriangleCount) + throws Exception { + output.f0 = vertexAndDegree.f0; + output.f1.f0 = vertexAndDegree.f1; + output.f1.f1 = (vertexAndTriangleCount == null) ? zero : vertexAndTriangleCount.f1; + + return output; + } + } + + /** + * Wraps the vertex type to encapsulate results from the Clustering Coefficient algorithm. + * + * @param ID type + */ + public static class Result + extends Vertex> { + private static final int HASH_SEED = 0xc23937c1; + + private Murmur3_32 hasher = new Murmur3_32(HASH_SEED); + + /** + * The no-arg constructor instantiates contained objects. + */ + public Result() { + f1 = new Tuple2<>(); + } + + /** + * Get the vertex degree. + * + * @return vertex degree + */ + public LongValue getDegree() { + return f1.f0; + } + + /** + * Get the number of triangles containing this vertex; equivalently, + * this is the number of edges between neighbors of this vertex. + * + * @return triangle count + */ + public LongValue getTriangleCount() { + return f1.f1; + } + + /** + * Get the local clustering coefficient score. This is computed as the + * number of edges between neighbors, equal to the triangle count, + * divided by the number of potential edges between neighbors. + * + * A score of {@code Double.NaN} is returned for a vertex with degree 1 + * for which both the triangle count and number of neighbors are zero. + * + * @return local clustering coefficient score + */ + public double getLocalClusteringCoefficientScore() { + long degree = getDegree().getValue(); + long neighborPairs = degree * (degree - 1) / 2; + + return (neighborPairs == 0) ? Double.NaN : getTriangleCount().getValue() / (double)neighborPairs; + } + + public String toVerboseString() { + return "Vertex ID: " + f0 + + ", vertex degree: " + getDegree() + + ", triangle count: " + getTriangleCount() + + ", local clustering coefficient: " + getLocalClusteringCoefficientScore(); + } + + @Override + public int hashCode() { + return hasher.reset() + .hash(f0.hashCode()) + .hash(f1.f0.getValue()) + .hash(f1.f1.getValue()) + .hash(); + } + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java new file mode 100644 index 0000000000000..e0ad30ff42db3 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/clustering/undirected/TriangleListing.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.library.clustering.undirected; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.operators.Order; +import org.apache.flink.api.common.operators.base.JoinOperatorBase; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; +import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.asm.degree.annotate.undirected.EdgeDegreePair; +import org.apache.flink.types.CopyableValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.util.Collector; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * Generates a listing of distinct triangles from the input graph. + *
+ * A triangle is a 3-cycle with vertices A, B, and C connected by edges + * (A, B), (A, C), and (B, C). + *
+ * The input graph must be a simple, undirected graph containing no duplicate + * edges or self-loops. + *
+ * Algorithm from "Graph Twiddling in a MapReduce World", J. D. Cohen, + * http://lintool.github.io/UMD-courses/bigdata-2015-Spring/content/Cohen_2009.pdf + * + * @param graph ID type + * @param vertex value type + * @param edge value type + */ +public class TriangleListing & CopyableValue, VV, EV> +implements GraphAlgorithm>> { + + // Optional configuration + private boolean sortTriangleVertices = false; + + private int littleParallelism = ExecutionConfig.PARALLELISM_UNKNOWN; + + /** + * Normalize the triangle listing such that for each result (K0, K1, K2) + * the vertex IDs are sorted K0 < K1 < K2. + * + * @param sortTriangleVertices whether to output each triangle's vertices in sorted order + * @return this + */ + public TriangleListing setSortTriangleVertices(boolean sortTriangleVertices) { + this.sortTriangleVertices = sortTriangleVertices; + + return this; + } + + /** + * Override the parallelism of operators processing small amounts of data. + * + * @param littleParallelism operator parallelism + * @return this + */ + public TriangleListing setLittleParallelism(int littleParallelism) { + this.littleParallelism = littleParallelism; + + return this; + } + + /* + * Implementation notes: + * + * The requirement that "K extends CopyableValue" can be removed when + * Flink has a self-join and GenerateTriplets is implemented as such. + * + * ProjectTriangles should eventually be replaced by ".projectFirst("*")" + * when projections use code generation. + */ + + @Override + public DataSet> run(Graph input) + throws Exception { + // u, v where u < v + DataSet> filteredByID = input + .getEdges() + .flatMap(new FilterByID()) + .setParallelism(littleParallelism) + .name("Filter by ID"); + + // u, v, (edge value, deg(u), deg(v)) + DataSet>> pairDegree = input + .run(new EdgeDegreePair() + .setParallelism(littleParallelism)); + + // u, v where deg(u) < deg(v) or (deg(u) == deg(v) and u < v) + DataSet> filteredByDegree = pairDegree + .flatMap(new FilterByDegree()) + .setParallelism(littleParallelism) + .name("Filter by degree"); + + // u, v, w where (u, v) and (u, w) are edges in graph, v < w + DataSet> triplets = filteredByDegree + .groupBy(0) + .sortGroup(1, Order.ASCENDING) + .reduceGroup(new GenerateTriplets()) + .setParallelism(littleParallelism) + .name("Generate triplets"); + + // u, v, w where (u, v), (u, w), and (v, w) are edges in graph, v < w + DataSet> triangles = triplets + .join(filteredByID, JoinOperatorBase.JoinHint.REPARTITION_HASH_SECOND) + .where(1, 2) + .equalTo(0, 1) + .with(new ProjectTriangles()) + .setParallelism(littleParallelism) + .name("Triangle listing"); + + if (sortTriangleVertices) { + triangles = triangles + .map(new SortTriangleVertices()) + .name("Sort triangle vertices"); + } + + return triangles; + } + + /** + * Removes edge values while filtering such that only edges where the + * source vertex ID compares less than the target vertex ID are emitted. + *
+ * Since the input graph is a simple graph this filter removes exactly half + * of the original edges. + * + * @param ID type + * @param edge value type + */ + @ForwardedFields("0; 1") + private static final class FilterByID, ET> + implements FlatMapFunction, Tuple2> { + private Tuple2 edge = new Tuple2<>(); + + @Override + public void flatMap(Edge value, Collector> out) + throws Exception { + if (value.f0.compareTo(value.f1) < 0) { + edge.f0 = value.f0; + edge.f1 = value.f1; + out.collect(edge); + } + } + } + + /** + * Removes edge values while filtering such that edges where the source + * vertex has lower degree are emitted. If the source and target vertex + * degrees are equal then the edge is emitted if the source vertex ID + * compares less than the target vertex ID. + *
+ * Since the input graph is a simple graph this filter removes exactly half + * of the original edges. + * + * @param ID type + */ + @ForwardedFields("0; 1") + private static final class FilterByDegree, ET> + implements FlatMapFunction>, Tuple2> { + private Tuple2 edge = new Tuple2<>(); + + @Override + public void flatMap(Edge> value, Collector> out) + throws Exception { + Tuple3 degrees = value.f2; + long sourceDegree = degrees.f1.getValue(); + long targetDegree = degrees.f2.getValue(); + + if (sourceDegree < targetDegree || + (sourceDegree == targetDegree && value.f0.compareTo(value.f1) < 0)) { + edge.f0 = value.f0; + edge.f1 = value.f1; + out.collect(edge); + } + } + } + + /** + * Generates the set of triplets by the pairwise enumeration of the open + * neighborhood for each vertex. The number of triplets is quadratic in + * the vertex degree; however, data skew is minimized by only generating + * triplets from the vertex with least degree. + * + * @param ID type + */ + private static final class GenerateTriplets> + implements GroupReduceFunction, Tuple3> { + private Tuple3 output = new Tuple3<>(); + + private List visited = new ArrayList<>(); + + @Override + public void reduce(Iterable> values, Collector> out) + throws Exception { + int visitedCount = 0; + + Iterator> iter = values.iterator(); + + while (true) { + Tuple2 edge = iter.next(); + + output.f0 = edge.f0; + output.f2 = edge.f1; + + for (int i = 0; i < visitedCount; i++) { + output.f1 = visited.get(i); + out.collect(output); + } + + if (! iter.hasNext()) { + break; + } + + if (visitedCount == visited.size()) { + visited.add(edge.f1.copy()); + } else { + edge.f1.copyTo(visited.get(visitedCount)); + } + + visitedCount += 1; + } + } + } + + /** + * Simply project the triplet as a triangle. + * + * @param ID type + */ + @ForwardedFieldsFirst("0; 1; 2") + @ForwardedFieldsSecond("0; 1") + private static final class ProjectTriangles + implements JoinFunction, Tuple2, Tuple3> { + @Override + public Tuple3 join(Tuple3 first, Tuple2 second) + throws Exception { + return first; + } + } + + /** + * Reorders the vertices of each emitted triangle (K0, K1, K2) + * into sorted order such that K0 < K1 < K2. + * + * @param ID type + */ + private static final class SortTriangleVertices> + implements MapFunction, Tuple3> { + @Override + public Tuple3 map(Tuple3 value) + throws Exception { + T temp_val; + + // by the triangle listing algorithm we know f1 < f2 + if (value.f0.compareTo(value.f1) > 0) { + temp_val = value.f0; + value.f0 = value.f1; + + if (temp_val.compareTo(value.f2) <= 0) { + value.f1 = temp_val; + } else { + value.f1 = value.f2; + value.f2 = temp_val; + } + } + + return value; + } + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Murmur3_32.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Murmur3_32.java new file mode 100644 index 0000000000000..98f2b97302c91 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/utils/Murmur3_32.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.utils; + +import java.io.Serializable; + +/** + * A resettable implementation of the 32-bit MurmurHash algorithm. + */ +public class Murmur3_32 implements Serializable { + + private static final long serialVersionUID = 1L; + + // initial seed, which can be reset + private final int seed; + + // number of 32-bit values processed + private int count; + + // in-progress hash value + private int hash; + + /** + * A resettable implementation of the 32-bit MurmurHash algorithm. + * + * @param seed MurmurHash seed + */ + public Murmur3_32(int seed) { + this.seed = seed; + reset(); + } + + /** + * Re-initialize the MurmurHash state. + * + * @return this + */ + public Murmur3_32 reset() { + count = 0; + hash = seed; + return this; + } + + /** + * Process an {@code integer} value. + * + * @param input 32-bit input value + * @return this + */ + public Murmur3_32 hash(int input) { + count++; + + input *= 0xcc9e2d51; + input = input << 15; + input *= 0x1b873593; + + hash ^= input; + hash = hash << 13; + hash = hash * 5 + 0xe6546b64; + + return this; + } + + /** + * Process a {@code long} value. + * + * @param input 64-bit input value + * @return this + */ + public Murmur3_32 hash(long input) { + hash((int)(input >>> 32)); + hash((int)input); + return this; + } + + /** + * Finalize and return the MurmurHash output. + * + * @return 32-bit hash + */ + public int hash() { + hash ^= 4 * count; + hash ^= hash >>> 16; + hash *= 0x85ebca6b; + hash ^= hash >>> 13; + hash *= 0xc2b2ae35; + hash ^= hash >>> 16; + + return hash; + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java new file mode 100644 index 0000000000000..414f200001fce --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/LocalClusteringCoefficientTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.library.clustering.undirected; + +import org.apache.commons.math3.util.CombinatoricsUtils; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.graph.library.clustering.undirected.LocalClusteringCoefficient.Result; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class LocalClusteringCoefficientTest +extends AsmTestBase { + + @Test + public void testSimpleGraph() + throws Exception { + DataSet> cc = undirectedSimpleGraph + .run(new LocalClusteringCoefficient()); + + String expectedResult = + "(0,(2,1))\n" + + "(1,(3,2))\n" + + "(2,(3,2))\n" + + "(3,(4,1))\n" + + "(4,(1,0))\n" + + "(5,(1,0))"; + + TestBaseUtils.compareResultAsText(cc.collect(), expectedResult); + } + + @Test + public void testCompleteGraph() + throws Exception { + long expectedDegree = completeGraphVertexCount - 1; + long expectedTriangleCount = CombinatoricsUtils.binomialCoefficient((int)expectedDegree, 2); + + DataSet> cc = completeGraph + .run(new LocalClusteringCoefficient()); + + for (Result result : cc.collect()) { + assertEquals(expectedDegree, result.getDegree().getValue()); + assertEquals(expectedTriangleCount, result.getTriangleCount().getValue()); + } + } + + @Test + public void testRMatGraph() + throws Exception { + DataSet> cc = undirectedRMatGraph + .run(new LocalClusteringCoefficient()); + + ChecksumHashCode checksum = DataSetUtils.checksumHashCode(cc); + + assertEquals(902, checksum.getCount()); + assertEquals(0x000001b08e783277L, checksum.getChecksum()); + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java new file mode 100644 index 0000000000000..0d1ebd013374b --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/clustering/undirected/TriangleListingTest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.library.clustering.undirected; + +import org.apache.commons.math3.util.CombinatoricsUtils; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.Utils.ChecksumHashCode; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.utils.DataSetUtils; +import org.apache.flink.graph.asm.AsmTestBase; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.IntValue; +import org.apache.flink.types.LongValue; +import org.apache.flink.types.NullValue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class TriangleListingTest +extends AsmTestBase { + + @Test + public void testSimpleGraph() + throws Exception { + DataSet> tl = undirectedSimpleGraph + .run(new TriangleListing() + .setSortTriangleVertices(true)); + + String expectedResult = + "(0,1,2)\n" + + "(1,2,3)"; + + TestBaseUtils.compareResultAsText(tl.collect(), expectedResult); + } + + @Test + public void testCompleteGraph() + throws Exception { + long expectedDegree = completeGraphVertexCount - 1; + long expectedCount = completeGraphVertexCount * CombinatoricsUtils.binomialCoefficient((int)expectedDegree, 2) / 3; + + DataSet> tl = completeGraph + .run(new TriangleListing()); + + ChecksumHashCode checksum = DataSetUtils.checksumHashCode(tl); + + assertEquals(expectedCount, checksum.getCount()); + } + + @Test + public void testRMatGraph() + throws Exception { + DataSet> tl = undirectedRMatGraph + .run(new TriangleListing() + .setSortTriangleVertices(true)); + + ChecksumHashCode checksum = DataSetUtils.checksumHashCode(tl); + + assertEquals(75049, checksum.getCount()); + assertEquals(0x00000001a5b500afL, checksum.getChecksum()); + } +} From 2056e3fe918ef24c377e314818aa4197b52a8abf Mon Sep 17 00:00:00 2001 From: Joshi Date: Tue, 17 May 2016 11:00:00 -0700 Subject: [PATCH 21/70] [FLINK-3488] [tests] Fix flakey test Kafka08ITCase.testBigRecordJob This closes #1998 --- .../flink/streaming/connectors/kafka/KafkaConsumerTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index cc9205cf91ac7..3ba8cff281042 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -840,7 +840,7 @@ public void invoke(Tuple2 value) throws Exception { // add producing topology Properties producerProps = new Properties(); - producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 14)); + producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 15)); producerProps.setProperty("retries", "3"); producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings); From bfd2c022f3c47288dae688c75c07101109a56ab9 Mon Sep 17 00:00:00 2001 From: Joshi Date: Mon, 16 May 2016 15:18:42 -0700 Subject: [PATCH 22/70] [FLINK-3782] [tests] Properly close streams in CollectionInputFormatTest This closes #1995 --- .../java/io/CollectionInputFormatTest.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java index ebaa44d420b76..53523b01d7d2a 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java @@ -80,7 +80,8 @@ public int hashCode() { @Test public void testSerializability() { - try { + try (ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(buffer)) { Collection inputCollection = new ArrayList(); ElementType element1 = new ElementType(1); ElementType element2 = new ElementType(2); @@ -95,9 +96,6 @@ public void testSerializability() { CollectionInputFormat inputFormat = new CollectionInputFormat(inputCollection, info.createSerializer(new ExecutionConfig())); - ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buffer); - out.writeObject(inputFormat); ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(buffer.toByteArray())); @@ -125,6 +123,7 @@ public void testSerializability() { e.printStackTrace(); fail(e.toString()); } + } @Test @@ -204,14 +203,12 @@ public void testSerializabilityStrings() { @Test public void testSerializationFailure() { - try { + try (ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(buffer)) { // a mock serializer that fails when writing CollectionInputFormat inFormat = new CollectionInputFormat( Collections.singleton(new ElementType()), new TestSerializer(false, true)); - ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buffer); - try { out.writeObject(inFormat); fail("should throw an exception"); @@ -231,13 +228,12 @@ public void testSerializationFailure() { @Test public void testDeserializationFailure() { - try { + try (ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(buffer)) { // a mock serializer that fails when writing CollectionInputFormat inFormat = new CollectionInputFormat( Collections.singleton(new ElementType()), new TestSerializer(true, false)); - - ByteArrayOutputStream buffer = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buffer); + out.writeObject(inFormat); out.close(); From 517664be4f64ae9353f075bac58e8cb0b6001529 Mon Sep 17 00:00:00 2001 From: markreddy Date: Wed, 11 May 2016 09:55:19 +0100 Subject: [PATCH 23/70] [FLINK-3852] [quickstart] Add skeleton StreamingJob - move Job to BatchJob - comment out transformers for the mainClass setting - tidy up SocketTextStreamWordCount - update docs This closes #1982 --- docs/quickstart/java_api_quickstart.md | 4 +- docs/quickstart/scala_api_quickstart.md | 4 +- .../resources/archetype-resources/pom.xml | 7 +- .../src/main/java/{Job.java => BatchJob.java} | 19 +++-- .../main/java/SocketTextStreamWordCount.java | 9 ++- .../src/main/java/StreamingJob.java | 73 +++++++++++++++++++ .../resources/archetype-resources/pom.xml | 7 +- .../main/scala/{Job.scala => BatchJob.scala} | 26 ++++--- .../scala/SocketTextStreamWordCount.scala | 10 +-- .../src/main/scala/StreamingJob.scala | 73 +++++++++++++++++++ 10 files changed, 198 insertions(+), 34 deletions(-) rename flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/{Job.java => BatchJob.java} (79%) create mode 100644 flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java rename flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/{Job.scala => BatchJob.scala} (76%) create mode 100644 flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md index 8b582bdd191ad..69a177ba16a70 100644 --- a/docs/quickstart/java_api_quickstart.md +++ b/docs/quickstart/java_api_quickstart.md @@ -63,9 +63,9 @@ Use one of the following commands to __create a project__: There will be a new directory in your working directory. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId. -The sample project is a __Maven project__, which contains two classes. _Job_ is a basic skeleton program and _WordCountJob_ a working example. Please note that the _main_ method of both classes allow you to start Flink in a development/testing mode. +The sample project is a __Maven project__, which contains four classes. _StreamingJob_ and _BatchJob_ are basic skeleton programs, _SocketTextStreamWordCount_ is a working streaming example and _WordCountJob_ is a working batch example. Please note that the _main_ method of all classes allow you to start Flink in a development/testing mode. -We recommend to __import this project into your IDE__ to develop and test it. If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import). Some Eclipse bundles include that plugin by default, others require you to install it manually. The IntelliJ IDE also supports Maven projects out of the box. +We recommend you __import this project into your IDE__ to develop and test it. If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import). Some Eclipse bundles include that plugin by default, others require you to install it manually. The IntelliJ IDE also supports Maven projects out of the box. A note to Mac OS X users: The default JVM heapsize for Java is too small for Flink. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse. diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md index 423409ad6cd67..b04922fe19e1a 100644 --- a/docs/quickstart/scala_api_quickstart.md +++ b/docs/quickstart/scala_api_quickstart.md @@ -154,9 +154,9 @@ $ curl https://flink.apache.org/q/quickstart-scala.sh | bash There will be a new directory in your working directory. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId. -The sample project is a __Maven project__, which contains two classes. _Job_ is a basic skeleton program and _WordCountJob_ a working example. Please note that the _main_ method of both classes allow you to start Flink in a development/testing mode. +The sample project is a __Maven project__, which contains four classes. _StreamingJob_ and _BatchJob_ are basic skeleton programs, _SocketTextStreamWordCount_ is a working streaming example and _WordCountJob_ is a working batch example. Please note that the _main_ method of all classes allow you to start Flink in a development/testing mode. -We recommend to __import this project into your IDE__. For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites: +We recommend you __import this project into your IDE__. For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites: * _Eclipse 4.x_ * [Scala IDE](http://download.scala-ide.org/sdk/e38/scala210/stable/site) 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 c9fb2ab9b6f30..5837c41bebb7c 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 @@ -256,12 +256,15 @@ under the License. + + - ${package}.Job + ${package}.StreamingJob + --> false diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/Job.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java similarity index 79% rename from flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/Job.java rename to flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java index 984dc2f8aa652..325ebae1b1d7f 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/Job.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/BatchJob.java @@ -21,9 +21,9 @@ import org.apache.flink.api.java.ExecutionEnvironment; /** - * Skeleton for a Flink Job. + * Skeleton for a Flink Batch Job. * - * For a full example of a Flink Job, see the WordCountJob.java file in the + * For a full example of a Flink Batch Job, see the WordCountJob.java file in the * same package/directory or have a look at the website. * * You can also generate a .jar file that you can submit on your Flink @@ -32,16 +32,20 @@ * mvn clean package * in the projects root directory. * You will find the jar in - * target/flink-quickstart-0.1-SNAPSHOT-Sample.jar + * target/flink-quickstart-${version}.jar + * From the CLI you can then run + * ./bin/flink run -c ${package}.BatchJob target/flink-quickstart-${version}.jar * + * For more information on the CLI see: + * + * http://flink.apache.org/docs/latest/apis/cli.html */ -public class Job { +public class BatchJob { public static void main(String[] args) throws Exception { - // set up the execution environment + // set up the batch execution environment final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - /** * Here, you can start creating your execution plan for Flink. * @@ -54,6 +58,7 @@ public static void main(String[] args) throws Exception { * .flatMap() * .join() * .coGroup() + * * and many more. * Have a look at the programming guide for the Java API: * @@ -66,6 +71,6 @@ public static void main(String[] args) throws Exception { */ // execute program - env.execute("Flink Java API Skeleton"); + env.execute("Flink Batch Java API Skeleton"); } } diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java index 6a3383c73cd18..abd62bb0c2ff3 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/SocketTextStreamWordCount.java @@ -36,15 +36,15 @@ * *

* Usage: - * SocketTextStreamWordCount <hostname> <port> <result path> + * SocketTextStreamWordCount <hostname> <port> *
* *

* This example shows how to: *

    *
  • use StreamExecutionEnvironment.socketTextStream - *
  • write a simple Flink program, - *
  • write and use user-defined functions. + *
  • write a simple Flink program + *
  • write and use user-defined functions *
* * @see netcat @@ -82,8 +82,9 @@ public static void main(String[] args) throws Exception { counts.print(); // execute program - env.execute("WordCount from SocketTextStream Example"); + env.execute("Java WordCount from SocketTextStream Example"); } + // // User Functions // diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java new file mode 100644 index 0000000000000..0d56fb60455f3 --- /dev/null +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/src/main/java/StreamingJob.java @@ -0,0 +1,73 @@ +package ${package}; + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + + +/** + * Skeleton for a Flink Streaming Job. + * + * For a full example of a Flink Streaming Job, see the SocketTextStreamWordCount.java + * file in the same package/directory or have a look at the website. + * + * You can also generate a .jar file that you can submit on your Flink + * cluster. + * Just type + * mvn clean package + * in the projects root directory. + * You will find the jar in + * target/flink-quickstart-${version}.jar + * From the CLI you can then run + * ./bin/flink run -c ${package}.StreamingJob target/flink-quickstart-${version}.jar + * + * For more information on the CLI see: + * + * http://flink.apache.org/docs/latest/apis/cli.html + */ +public class StreamingJob { + + public static void main(String[] args) throws Exception { + // set up the streaming execution environment + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + /** + * Here, you can start creating your execution plan for Flink. + * + * Start with getting some data from the environment, like + * env.readTextFile(textPath); + * + * then, transform the resulting DataStream using operations + * like + * .filter() + * .flatMap() + * .join() + * .coGroup() + * + * and many more. + * Have a look at the programming guide for the Java API: + * + * http://flink.apache.org/docs/latest/apis/streaming/index.html + * + */ + + // execute program + env.execute("Flink Streaming Java API Skeleton"); + } +} 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 4752e6f902615..d64cc128cbd86 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 @@ -259,12 +259,15 @@ under the License. + + - ${package}.Job + ${package}.StreamingJob + --> false diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala similarity index 76% rename from flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala rename to flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala index 45fcbcb4df987..d0d4d26715dc5 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/Job.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/BatchJob.scala @@ -21,9 +21,9 @@ package ${package} import org.apache.flink.api.scala._ /** - * Skeleton for a Flink Job. + * Skeleton for a Flink Batch Job. * - * For a full example of a Flink Job, see the WordCountJob.scala file in the + * For a full example of a Flink Batch Job, see the WordCountJob.scala file in the * same package/directory or have a look at the website. * * You can also generate a .jar file that you can submit on your Flink @@ -32,22 +32,29 @@ import org.apache.flink.api.scala._ * mvn clean package * }}} * in the projects root directory. You will find the jar in - * target/flink-quickstart-0.1-SNAPSHOT-Sample.jar + * target/flink-quickstart-${version}.jar + * From the CLI you can then run + * {{{ + * ./bin/flink run -c ${package}.BatchJob target/flink-quickstart-${version}.jar + * }}} * + * For more information on the CLI see: + * + * http://flink.apache.org/docs/latest/apis/cli.html */ -object Job { +object BatchJob { def main(args: Array[String]) { - // set up the execution environment + // set up the batch execution environment val env = ExecutionEnvironment.getExecutionEnvironment /** * Here, you can start creating your execution plan for Flink. * * Start with getting some data from the environment, like - * env.readTextFile(textPath); + * env.readTextFile(textPath); * * then, transform the resulting DataSet[String] using operations - * like: + * like * .filter() * .flatMap() * .join() @@ -64,8 +71,7 @@ object Job { * */ - // execute program - env.execute("Flink Scala API Skeleton") + env.execute("Flink Batch Scala API Skeleton") } -} +} \ No newline at end of file diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala index d08974dd0a831..a6987acc613b9 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/SocketTextStreamWordCount.scala @@ -33,14 +33,14 @@ import org.apache.flink.streaming.api.scala._ * * Usage: * {{{ - * SocketTextStreamWordCount + * SocketTextStreamWordCount * }}} * * This example shows how to: * * - use StreamExecutionEnvironment.socketTextStream - * - write a simple Flink Streaming program in scala. - * - write and use user-defined functions. + * - write a simple Flink Streaming program in scala + * - write and use user-defined functions */ object SocketTextStreamWordCount { @@ -55,7 +55,7 @@ object SocketTextStreamWordCount { val env = StreamExecutionEnvironment.getExecutionEnvironment - //Create streams for names and ages by mapping the inputs to the corresponding objects + // create streams for names and ages by mapping the inputs to the corresponding objects val text = env.socketTextStream(hostName, port) val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } } .map { (_, 1) } @@ -64,6 +64,6 @@ object SocketTextStreamWordCount { counts print - env.execute("Scala SocketTextStreamWordCount Example") + env.execute("Scala WordCount from SocketTextStream Example") } } diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala new file mode 100644 index 0000000000000..cb1ad3c562a2b --- /dev/null +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/src/main/scala/StreamingJob.scala @@ -0,0 +1,73 @@ +package ${package} + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.flink.api.scala._ + +/** + * Skeleton for a Flink Streaming Job. + * + * For a full example of a Flink Streaming Job, see the SocketTextStreamWordCount.java + * file in the same package/directory or have a look at the website. + * + * You can also generate a .jar file that you can submit on your Flink + * cluster. Just type + * {{{ + * mvn clean package + * }}} + * in the projects root directory. You will find the jar in + * target/flink-quickstart-${version}.jar + * From the CLI you can then run + * {{{ + * ./bin/flink run -c ${package}.StreamingJob target/flink-quickstart-${version}.jar + * }}} + * + * For more information on the CLI see: + * + * http://flink.apache.org/docs/latest/apis/cli.html + */ +object StreamingJob { + def main(args: Array[String]) { + // set up the streaming execution environment + val env = StreamExecutionEnvironment.getExecutionEnvironment + + /** + * Here, you can start creating your execution plan for Flink. + * + * Start with getting some data from the environment, like + * env.readTextFile(textPath); + * + * then, transform the resulting DataStream[String] using operations + * like + * .filter() + * .flatMap() + * .join() + * .group() + * + * and many more. + * Have a look at the programming guide: + * + * http://flink.apache.org/docs/latest/apis/streaming/index.html + * + */ + + // execute program + env.execute("Flink Streaming Scala API Skeleton") + } +} \ No newline at end of file From e8455bf83db9874f507c5df32f174624f99c925c Mon Sep 17 00:00:00 2001 From: kl0u Date: Tue, 10 May 2016 16:00:03 +0200 Subject: [PATCH 24/70] [FLINK-3826] [tests] Fix test condition in StreamCheckpointingITCase This closes #1977 --- .../flink/test/checkpointing/StreamCheckpointingITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java index 27c0f80a88c12..b97e1f26ef231 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java @@ -267,7 +267,7 @@ public PrefixCount map(PrefixCount value) throws Exception { count++; if (!hasFailed && count >= failurePos) { hasFailed = true; -// throw new Exception("Test Failure"); + throw new Exception("Test Failure"); } inputCount++; From 180662ad35f878efdf8b30ec2464c730cb866417 Mon Sep 17 00:00:00 2001 From: Steve Cosenza Date: Sun, 8 May 2016 16:55:06 -0700 Subject: [PATCH 25/70] [hotfix] [kafka tests] Build a flink-connector-kafka-0.9 test-jar Build a test-jar including KafkaTestEnvironmentImpl so that Flink users can write end to end integration tests of their Flink jobs. This closes #1972 --- .../flink-connector-kafka-0.9/pom.xml | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml index be40233226d8b..f0bd98ef899ce 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml @@ -113,6 +113,39 @@ under the License. + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-test-sources + + test-jar-no-fork + + + + **/KafkaTestEnvironmentImpl* + + + + + org.apache.maven.plugins maven-surefire-plugin From c3e37ad48993d41afddd1c94ddcf17f55c85e137 Mon Sep 17 00:00:00 2001 From: Mark Reddy Date: Mon, 16 May 2016 00:52:53 +0100 Subject: [PATCH 26/70] [FLINK-3913] [docs] Clean up spelling mistakes. This closes #1992 --- docs/apis/batch/dataset_transformations.md | 4 ++-- docs/apis/batch/examples.md | 6 +++--- docs/apis/batch/index.md | 16 +++++++-------- docs/apis/batch/iterations.md | 2 +- docs/apis/batch/libs/gelly.md | 4 ++-- .../libs/ml/multiple_linear_regression.md | 2 +- docs/apis/batch/libs/ml/pipelines.md | 2 +- docs/apis/batch/python.md | 2 +- docs/apis/best_practices.md | 2 +- docs/apis/common/index.md | 4 ++-- docs/apis/scala_shell.md | 2 +- docs/apis/streaming/connectors/rabbitmq.md | 2 +- docs/apis/streaming/event_time.md | 2 +- .../streaming/event_timestamps_watermarks.md | 2 +- docs/apis/streaming/fault_tolerance.md | 2 +- docs/apis/streaming/index.md | 20 +++++++++---------- docs/apis/streaming/storm_compatibility.md | 4 ++-- docs/apis/table.md | 6 +++--- docs/internals/back_pressure_monitoring.md | 2 +- docs/internals/general_arch.md | 2 +- docs/internals/job_scheduling.md | 4 ++-- docs/internals/monitoring_rest_api.md | 6 +++--- docs/setup/config.md | 8 ++++---- docs/setup/local_setup.md | 2 +- docs/setup/yarn_setup.md | 8 ++++---- 25 files changed, 58 insertions(+), 58 deletions(-) diff --git a/docs/apis/batch/dataset_transformations.md b/docs/apis/batch/dataset_transformations.md index 2d7281717e1ed..0de771a34561f 100644 --- a/docs/apis/batch/dataset_transformations.md +++ b/docs/apis/batch/dataset_transformations.md @@ -1514,7 +1514,7 @@ The following hints are available: partitioned) and builds a hash table from the first input. This strategy is good if the first input is smaller than the second, but both inputs are still large. *Note:* This is the default fallback strategy that the system uses if no size estimates can be made - and no pre-existing partitiongs and sort-orders can be re-used. + and no pre-existing partitions and sort-orders can be re-used. * `REPARTITION_HASH_SECOND`: The system partitions (shuffles) each input (unless the input is already partitioned) and builds a hash table from the second input. This strategy is good if the second @@ -1528,7 +1528,7 @@ The following hints are available: ### OuterJoin -The OuterJoin transformation performs a left, right, or full outer join on two data sets. Outer joins are similar to regular (inner) joins and create all pairs of elements that are equal on their keys. In addition, records of the "outer" side (left, right, or both in case of full) are preserved if no matching key is found in the other side. Matching pair of elements (or one element and a `null` value for the other input) are given to a `JoinFunction` to turn the pair of elements into a single element, or to a `FlatJoinFunction` to turn the pair of elements into arbitararily many (including none) elements. +The OuterJoin transformation performs a left, right, or full outer join on two data sets. Outer joins are similar to regular (inner) joins and create all pairs of elements that are equal on their keys. In addition, records of the "outer" side (left, right, or both in case of full) are preserved if no matching key is found in the other side. Matching pair of elements (or one element and a `null` value for the other input) are given to a `JoinFunction` to turn the pair of elements into a single element, or to a `FlatJoinFunction` to turn the pair of elements into arbitrarily many (including none) elements. The elements of both DataSets are joined on one or more keys which can be specified using diff --git a/docs/apis/batch/examples.md b/docs/apis/batch/examples.md index 3b6ccd4abcf38..35231854d1015 100644 --- a/docs/apis/batch/examples.md +++ b/docs/apis/batch/examples.md @@ -118,7 +118,7 @@ val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } } counts.writeAsCsv(outputPath, "\n", " ") ~~~ -The {% gh_link /flink-examples/flink-exampls-batch/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala "WordCount example" %} implements the above described algorithm with input parameters: `--input --output `. As test data, any text file will do. +The {% gh_link /flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala "WordCount example" %} implements the above described algorithm with input parameters: `--input --output `. As test data, any text file will do.
@@ -171,11 +171,11 @@ public static final class JoinVertexWithEdgesMatch @Override public void join( page, Tuple2 adj, Collector> out) { - Long[] neigbors = adj.f1; + Long[] neighbors = adj.f1; double rank = page.f1; double rankToDistribute = rank / ((double) neigbors.length); - for (int i = 0; i < neigbors.length; i++) { + for (int i = 0; i < neighbors.length; i++) { out.collect(new Tuple2(neighbors[i], rankToDistribute)); } } diff --git a/docs/apis/batch/index.md b/docs/apis/batch/index.md index 653e5d7c3a2fe..4c0aea8472b01 100644 --- a/docs/apis/batch/index.md +++ b/docs/apis/batch/index.md @@ -348,7 +348,7 @@ DataSet data1 = // [...] DataSet data2 = // [...] DataSet> result = data1.cross(data2); {% endhighlight %} -

Note: Cross is potentially a very compute-intensive operation which can challenge even large compute clusters! It is adviced to hint the system with the DataSet sizes by using crossWithTiny() and crossWithHuge().

+

Note: Cross is potentially a very compute-intensive operation which can challenge even large compute clusters! It is advised to hint the system with the DataSet sizes by using crossWithTiny() and crossWithHuge().

@@ -560,7 +560,7 @@ val output: DataSet[(Int, String, Doublr)] = input.aggregate(SUM, 0).aggregate(M

You can also use short-hand syntax for minimum, maximum, and sum aggregations.

{% highlight scala %} val input: DataSet[(Int, String, Double)] = // [...] -val output: DataSet[(Int, String, Doublr)] = input.sum(0).min(2) +val output: DataSet[(Int, String, Double)] = input.sum(0).min(2) {% endhighlight %} @@ -1100,7 +1100,7 @@ DataSet: - `writeAsCsv(...)` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects. - `print()` / `printToErr()` / `print(String msg)` / `printToErr(String msg)` - Prints the *toString()* value -of each element on the standard out / strandard error stream. Optionally, a prefix (msg) can be provided which is +of each element on the standard out / standard error stream. Optionally, a prefix (msg) can be provided which is prepended to the output. This can help to distinguish between different calls to *print*. If the parallelism is greater than 1, the output will also be prepended with the identifier of the task which produced the output. - `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports @@ -1135,7 +1135,7 @@ values.writeAsCsv("file:///path/to/the/result/file", "\n", "|"); // this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines values.writeAsText("file:///path/to/the/result/file"); -// this wites values as strings using a user-defined TextFormatter object +// this writes values as strings using a user-defined TextFormatter object values.writeAsFormattedText("file:///path/to/the/result/file", new TextFormatter>() { public String format (Tuple2 value) { @@ -1199,12 +1199,12 @@ using an Flink comes with a variety of built-in output formats that are encapsulated behind operations on the DataSet: -- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are +- `writeAsText()` / `TextOutputFormat` - Writes elements line-wise as Strings. The Strings are obtained by calling the *toString()* method of each element. - `writeAsCsv(...)` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects. - `print()` / `printToErr()` - Prints the *toString()* value of each element on the - standard out / strandard error stream. + standard out / standard error stream. - `write()` / `FileOutputFormat` - Method and base class for custom file outputs. Supports custom object-to-bytes conversion. - `output()`/ `OutputFormat` - Most generic output method, for data sinks that are not file based @@ -1237,7 +1237,7 @@ values.writeAsCsv("file:///path/to/the/result/file", "\n", "|") // this writes tuples in the text formatting "(a, b, c)", rather than as CSV lines values.writeAsText("file:///path/to/the/result/file"); -// this wites values as strings using a user-defined formatting +// this writes values as strings using a user-defined formatting values map { tuple => tuple._1 + " - " + tuple._2 } .writeAsText("file:///path/to/the/result/file") {% endhighlight %} @@ -1616,7 +1616,7 @@ result data. This section give some hints how to ease the development of Flink p ### Local Execution Environment A `LocalEnvironment` starts a Flink system within the same JVM process it was created in. If you -start the LocalEnvironement from an IDE, you can set breakpoints in your code and easily debug your +start the LocalEnvironment from an IDE, you can set breakpoints in your code and easily debug your program. A LocalEnvironment is created and used as follows: diff --git a/docs/apis/batch/iterations.md b/docs/apis/batch/iterations.md index 912f378daae6c..bad058fa86ae2 100644 --- a/docs/apis/batch/iterations.md +++ b/docs/apis/batch/iterations.md @@ -191,7 +191,7 @@ In the following example, every vertex has an **ID** and a **coloring**. Each ve Delta Iterate Operator Example

-The **intial input** is set as **both workset and solution set.** In the above figure, the colors visualize the **evolution of the solution set**. With each iteration, the color of the minimum ID is spreading in the respective subgraph. At the same time, the amount of work (exchanged and compared vertex IDs) decreases with each iteration. This corresponds to the **decreasing size of the workset**, which goes from all seven vertices to zero after three iterations, at which time the iteration terminates. The **important observation** is that *the lower subgraph converges before the upper half* does and the delta iteration is able to capture this with the workset abstraction. +The **initial input** is set as **both workset and solution set.** In the above figure, the colors visualize the **evolution of the solution set**. With each iteration, the color of the minimum ID is spreading in the respective subgraph. At the same time, the amount of work (exchanged and compared vertex IDs) decreases with each iteration. This corresponds to the **decreasing size of the workset**, which goes from all seven vertices to zero after three iterations, at which time the iteration terminates. The **important observation** is that *the lower subgraph converges before the upper half* does and the delta iteration is able to capture this with the workset abstraction. In the upper subgraph **ID 1** (*orange*) is the **minimum ID**. In the **first iteration**, it will get propagated to vertex 2, which will subsequently change its color to orange. Vertices 3 and 4 will receive **ID 2** (in *yellow*) as their current minimum ID and change to yellow. Because the color of *vertex 1* didn't change in the first iteration, it can be skipped it in the next workset. diff --git a/docs/apis/batch/libs/gelly.md b/docs/apis/batch/libs/gelly.md index dd0b4c185cc9e..5d9bfd688d8e4 100644 --- a/docs/apis/batch/libs/gelly.md +++ b/docs/apis/batch/libs/gelly.md @@ -1957,7 +1957,7 @@ PageRank is an algorithm that was first used to rank web search engine results. #### Details The algorithm operates in iterations, where pages distribute their scores to their neighbors (pages they have links to) and subsequently update their scores based on the partial values they receive. The implementation assumes that each page has at least one incoming and one outgoing link. -In order to consider the importance of a link from one page to another, scores are divided by the total number of out-links of the source page. Thus, a page with 10 links will distribute 1/10 of its score to each neighbor, while a page with 100 links, will distribute 1/100 of its score to each neighboring page. This process computes what is often called the transition probablities, i.e. the probability that some page will lead to other page while surfing the web. To correctly compute the transition probabilities, this implementation expectes the edge values to be initialiez to 1.0. +In order to consider the importance of a link from one page to another, scores are divided by the total number of out-links of the source page. Thus, a page with 10 links will distribute 1/10 of its score to each neighbor, while a page with 100 links, will distribute 1/100 of its score to each neighboring page. This process computes what is often called the transition probablities, i.e. the probability that some page will lead to other page while surfing the web. To correctly compute the transition probabilities, this implementation expects the edge values to be initialised to 1.0. #### Usage The algorithm takes as input a `Graph` with any vertex type, `Double` vertex values, and `Double` edge values. Edges values should be initialized to 1.0, in order to correctly compute the transition probabilities. Otherwise, the transition probability for an Edge `(u, v)` will be set to the edge value divided by `u`'s out-degree. The algorithm returns a `DataSet` of vertices, where the vertex value corresponds to assigned rank after convergence (or maximum iterations). @@ -2679,7 +2679,7 @@ A directed or undirected power-law graph generated using the [Recursive Matrix (R-Mat)](http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf) model. RMat is a stochastic generator configured with a source of randomness implementing the -`RandomGenerableFactory` interface. Provided implemenations are `JDKRandomGeneratorFactory` +`RandomGenerableFactory` interface. Provided implementations are `JDKRandomGeneratorFactory` and `MersenneTwisterFactory`. These generate an initial sequence of random values which are then used as seeds for generating the edges. diff --git a/docs/apis/batch/libs/ml/multiple_linear_regression.md b/docs/apis/batch/libs/ml/multiple_linear_regression.md index 436f95b71f693..b427eac8eb588 100644 --- a/docs/apis/batch/libs/ml/multiple_linear_regression.md +++ b/docs/apis/batch/libs/ml/multiple_linear_regression.md @@ -135,7 +135,7 @@ the algorithm's performance.

Learning rate method used to calculate the effective learning rate for each iteration. - See the list of supported learing rate methods. + See the list of supported learning rate methods. (Default value: LearningRateMethod.Default)

diff --git a/docs/apis/batch/libs/ml/pipelines.md b/docs/apis/batch/libs/ml/pipelines.md index a6cce45adaa4e..f86476cd71982 100644 --- a/docs/apis/batch/libs/ml/pipelines.md +++ b/docs/apis/batch/libs/ml/pipelines.md @@ -403,7 +403,7 @@ Furthermore, we can now use this transformer as part of an analysis pipeline whe val trainingData: DataSet[DenseVector] = ... val mean = MeanTransformer.setMean(1.0) -val polyFeaturs = PolynomialFeatures().setDegree(3) +val polyFeatures = PolynomialFeatures().setDegree(3) val pipeline = mean.chainTransformer(polyFeatures) diff --git a/docs/apis/batch/python.md b/docs/apis/batch/python.md index 646683d47cd89..8771cb58472f2 100644 --- a/docs/apis/batch/python.md +++ b/docs/apis/batch/python.md @@ -563,7 +563,7 @@ The degree of parallelism of a task can be specified in Flink on different level ### Execution Environment Level -Flink programs are executed in the context of an [execution environmentt](#program-skeleton). An +Flink programs are executed in the context of an [execution environment](#program-skeleton). An execution environment defines a default parallelism for all operators, data sources, and data sinks it executes. Execution environment parallelism can be overwritten by explicitly configuring the parallelism of an operator. diff --git a/docs/apis/best_practices.md b/docs/apis/best_practices.md index 4d911dd58b0dd..62e0ebf434b53 100644 --- a/docs/apis/best_practices.md +++ b/docs/apis/best_practices.md @@ -214,7 +214,7 @@ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.getConfig().registerTypeWithKryoSerializer(MyCustomType.class, ProtobufSerializer.class); // register the serializer included with Apache Thrift as the standard serializer -// TBaseSerializer states it should be initalized as a default Kryo serializer +// TBaseSerializer states it should be initialized as a default Kryo serializer env.getConfig().addDefaultKryoSerializer(MyCustomType.class, TBaseSerializer.class); {% endhighlight %} diff --git a/docs/apis/common/index.md b/docs/apis/common/index.md index 049c5b39a6a19..9d06f12ebee0c 100644 --- a/docs/apis/common/index.md +++ b/docs/apis/common/index.md @@ -336,7 +336,7 @@ print() Once you specified the complete program you need to **trigger the program execution** by calling -`execute()` on the `StreamExecutionEnviroment`. +`execute()` on the `StreamExecutionEnvironment`. Depending on the type of the `ExecutionEnvironment` the execution will be triggered on your local machine or submit your program for execution on a cluster. @@ -1236,7 +1236,7 @@ Client can either be a Java or a Scala program. One example of such a Client is Flink's Command-line Interface (CLI). For the CLI client, the parallelism parameter can be specified with `-p`. For -exampple: +example: ./bin/flink run -p 10 ../examples/*WordCount-java*.jar diff --git a/docs/apis/scala_shell.md b/docs/apis/scala_shell.md index 0377f5a4aeaa9..ad36ca0810067 100644 --- a/docs/apis/scala_shell.md +++ b/docs/apis/scala_shell.md @@ -62,7 +62,7 @@ Scala-Flink> counts.print() The print() command will automatically send the specified tasks to the JobManager for execution and will show the result of the computation in the terminal. -It is possbile to write results to a file. However, in this case you need to call `execute`, to run your program: +It is possible to write results to a file. However, in this case you need to call `execute`, to run your program: ~~~scala Scala-Flink> benv.execute("MyProgram") diff --git a/docs/apis/streaming/connectors/rabbitmq.md b/docs/apis/streaming/connectors/rabbitmq.md index f5ea702575024..b48608d8fbfe5 100644 --- a/docs/apis/streaming/connectors/rabbitmq.md +++ b/docs/apis/streaming/connectors/rabbitmq.md @@ -50,7 +50,7 @@ The followings have to be provided for the `RMQSource(…)` constructor in order - hostName: The RabbitMQ broker hostname. - queueName: The RabbitMQ queue name. - usesCorrelationId: `true` when correlation ids should be used, `false` otherwise (default is `false`). -- deserializationScehma: Deserialization schema to turn messages into Java objects. +- deserializationSchema: Deserialization schema to turn messages into Java objects. This source can be operated in three different modes: diff --git a/docs/apis/streaming/event_time.md b/docs/apis/streaming/event_time.md index 0b4da03f493ec..7bc53e72f0406 100644 --- a/docs/apis/streaming/event_time.md +++ b/docs/apis/streaming/event_time.md @@ -84,7 +84,7 @@ Flink supports different notions of *time* in streaming programs. The first part of a Flink DataStream program is usually to set the base *time characteristic*. That setting defines how data stream sources behave (for example whether to assign timestamps), and what notion of -time the window operations like `KeyedStream.timeWindow(Time.secondss(30))` refer to. +time the window operations like `KeyedStream.timeWindow(Time.seconds(30))` refer to. The following example shows a Flink program that aggregates events in hourly time windows. The behavior of the windows adapts with the time characteristic. diff --git a/docs/apis/streaming/event_timestamps_watermarks.md b/docs/apis/streaming/event_timestamps_watermarks.md index 11c97a7893763..0ec2810620ea4 100644 --- a/docs/apis/streaming/event_timestamps_watermarks.md +++ b/docs/apis/streaming/event_timestamps_watermarks.md @@ -69,7 +69,7 @@ There are two ways to assign timestamps and generate Watermarks: Stream sources can also directly assign timestamps to the elements they produce and emit Watermarks. In that case, no Timestamp Assigner is needed. -To assign a timestamp to an element in the soruce directly, the source must use the `collectWithTimestamp(...)` +To assign a timestamp to an element in the source directly, the source must use the `collectWithTimestamp(...)` method on the `SourceContext`. To generate Watermarks, the source must call the `emitWatermark(Watermark)` function. Below is a simple example of a source *(non-checkpointed)* that assigns timestamps and generates Watermarks diff --git a/docs/apis/streaming/fault_tolerance.md b/docs/apis/streaming/fault_tolerance.md index d85038b6f6f03..f6e90a44f1de2 100644 --- a/docs/apis/streaming/fault_tolerance.md +++ b/docs/apis/streaming/fault_tolerance.md @@ -36,7 +36,7 @@ transient program failures, etc. Streaming Fault Tolerance ------------------------- -Flink has a checkpointing mechanism that recovers streaming jobs after failues. The checkpointing mechanism requires a *persistent* (or *durable*) source that +Flink has a checkpointing mechanism that recovers streaming jobs after failures. The checkpointing mechanism requires a *persistent* (or *durable*) source that can be asked for prior records again (Apache Kafka is a good example of such a source). The checkpointing mechanism stores the progress in the data sources and data sinks, the state of windows, as well as the user-defined state (see [Working with State](state.html)) consistently to provide *exactly once* processing semantics. Where the checkpoints are stored (e.g., JobManager memory, file system, database) depends on the configured [state backend](state_backends.html). diff --git a/docs/apis/streaming/index.md b/docs/apis/streaming/index.md index 17d8b561dbb81..8a825dd310ac8 100644 --- a/docs/apis/streaming/index.md +++ b/docs/apis/streaming/index.md @@ -270,7 +270,7 @@ DataStream result = Aggregations
KeyedStream → DataStream

Rolling aggregations on a keyed data stream. The difference between min - and minBy is that min returns the minimun value, whereas minBy returns + and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy).

{% highlight java %} keyedStream.sum(0); @@ -726,7 +726,7 @@ val result: DataStream[String] = Aggregations on windows
WindowedStream → DataStream

Aggregates the contents of a window. The difference between min - and minBy is that min returns the minimun value, whereas minBy returns + and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy).

{% highlight scala %} windowedStream.sum(0) @@ -1415,14 +1415,14 @@ Data sinks consume DataStreams and forward them to files, sockets, external syst Flink comes with a variety of built-in output formats that are encapsulated behind operations on the DataStreams: -- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are +- `writeAsText()` / `TextOutputFormat` - Writes elements line-wise as Strings. The Strings are obtained by calling the *toString()* method of each element. - `writeAsCsv(...)` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects. - `print()` / `printToErr()` - Prints the *toString()* value -of each element on the standard out / strandard error stream. Optionally, a prefix (msg) can be provided which is +of each element on the standard out / standard error stream. Optionally, a prefix (msg) can be provided which is prepended to the output. This can help to distinguish between different calls to *print*. If the parallelism is greater than 1, the output will also be prepended with the identifier of the task which produced the output. @@ -1443,14 +1443,14 @@ Data sinks consume DataStreams and forward them to files, sockets, external syst Flink comes with a variety of built-in output formats that are encapsulated behind operations on the DataStreams: -- `writeAsText()` / `TextOuputFormat` - Writes elements line-wise as Strings. The Strings are +- `writeAsText()` / `TextOutputFormat` - Writes elements line-wise as Strings. The Strings are obtained by calling the *toString()* method of each element. - `writeAsCsv(...)` / `CsvOutputFormat` - Writes tuples as comma-separated value files. Row and field delimiters are configurable. The value for each field comes from the *toString()* method of the objects. - `print()` / `printToErr()` - Prints the *toString()* value -of each element on the standard out / strandard error stream. Optionally, a prefix (msg) can be provided which is +of each element on the standard out / standard error stream. Optionally, a prefix (msg) can be provided which is prepended to the output. This can help to distinguish between different calls to *print*. If the parallelism is greater than 1, the output will also be prepended with the identifier of the task which produced the output. @@ -1472,7 +1472,7 @@ OutputFormat. This means that not all elements send to the OutputFormat are imme in the target system. Also, in failure cases, those records might be lost. For reliable, exactly-once delivery of a stream into a file system, use the `flink-connector-filesystem`. -Also, custom implementations through the `.addSink(...)` method can partiticpate in Flink's checkpointing +Also, custom implementations through the `.addSink(...)` method can participate in Flink's checkpointing for exactly-once semantics. {% top %} @@ -1494,7 +1494,7 @@ or a `filter`. Here, we show an example using filters. First, we define an `Iter IterativeStream iteration = input.iterate(); {% endhighlight %} -Then, we specify the logic that will be executed inside the loop using a series of trasformations (here +Then, we specify the logic that will be executed inside the loop using a series of transformations (here a simple `map` transformation) {% highlight java %} @@ -1627,7 +1627,7 @@ Usage: LocalStreamEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.setBufferTimeout(timeoutMillis); -env.genereateSequence(1,10).map(new MyMapper()).setBufferTimeout(timeoutMillis); +env.generateSequence(1,10).map(new MyMapper()).setBufferTimeout(timeoutMillis); {% endhighlight %}
@@ -1660,7 +1660,7 @@ result data. This section give some hints how to ease the development of Flink p ### Local Execution Environment A `LocalStreamEnvironment` starts a Flink system within the same JVM process it was created in. If you -start the LocalEnvironement from an IDE, you can set breakpoints in your code and easily debug your +start the LocalEnvironment from an IDE, you can set breakpoints in your code and easily debug your program. A LocalEnvironment is created and used as follows: diff --git a/docs/apis/streaming/storm_compatibility.md b/docs/apis/streaming/storm_compatibility.md index 6c8c0184e65ae..94e00428387ae 100644 --- a/docs/apis/streaming/storm_compatibility.md +++ b/docs/apis/streaming/storm_compatibility.md @@ -166,7 +166,7 @@ Bolts can accesses input tuple fields via name (additionally to access via index To use this feature with embedded Bolts, you need to have either a 1. [POJO]({{site.baseurl}}/apis/batch/index.html#pojos) type input stream or - 2. [Tuple]({{site.baseurl}}/apis/batch/index.html#tuples-and-case-classes) type input stream and spedify the input schema (ie, name-to-index-mapping) + 2. [Tuple]({{site.baseurl}}/apis/batch/index.html#tuples-and-case-classes) type input stream and specify the input schema (i.e. name-to-index-mapping) For POJO input types, Flink accesses the fields via reflection. For this case, Flink expects either a corresponding public member variable or public getter method. @@ -261,7 +261,7 @@ An example of a finite Spout that emits records for 10 seconds only:
~~~java public class TimedFiniteSpout extends BaseRichSpout implements FiniteSpout { - [...] // implemente open(), nextTuple(), ... + [...] // implement open(), nextTuple(), ... private long starttime = System.currentTimeMillis(); diff --git a/docs/apis/table.md b/docs/apis/table.md index 3defdbbd077e2..eae4ebd88bd34 100644 --- a/docs/apis/table.md +++ b/docs/apis/table.md @@ -29,7 +29,7 @@ under the License. **Table API and SQL are experimental features** The Table API is a SQL-like expression language for relational stream and batch processing that can be easily embedded in Flink's DataSet and DataStream APIs (Java and Scala). -The Table API and SQL insterface operate on a relational `Table` abstraction, which can be created from external data sources, or existing DataSets and DataStreams. With the Table API, you can apply relational operators such as selection, aggregation, and joins on `Table`s. +The Table API and SQL interface operate on a relational `Table` abstraction, which can be created from external data sources, or existing DataSets and DataStreams. With the Table API, you can apply relational operators such as selection, aggregation, and joins on `Table`s. `Table`s can also be queried with regular SQL, as long as they are registered (see [Registering and Accessing Tables](#registering-and-accessing-tables)). The Table API and SQL offer equivalent functionality and can be mixed in the same program. When a `Table` is converted back into a `DataSet` or `DataStream`, the logical plan, which was defined by relational operators and SQL queries, is optimized using [Apache Calcite](https://calcite.apache.org/) and transformed into a `DataSet` or `DataStream` execution plan. @@ -130,7 +130,7 @@ tableEnv.registerDataStream("Orders", ord, 'user, 'product, 'amount) ### Register a Table -A `Table` that originates from a Table API operation or a SQL query is registered in a `TableEnvironemnt` as follows: +A `Table` that originates from a Table API operation or a SQL query is registered in a `TableEnvironment` as follows:
@@ -688,7 +688,7 @@ SQL queries can be executed on DataStream Tables by adding the `STREAM` SQL keyw Emit a Table to external sinks ---- -A `Table` can be emitted to a `TableSink`, which is a generic interface to support a wide variaty of file formats (e.g., CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ), and others. A batch `Table` can only be emitted by a `BatchTableSink`, a streaming table requires a `StreamTableSink` (a `TableSink` can implement both interfaces). +A `Table` can be emitted to a `TableSink`, which is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ), and others. A batch `Table` can only be emitted by a `BatchTableSink`, a streaming table requires a `StreamTableSink` (a `TableSink` can implement both interfaces). Currently, Flink only provides a `CsvTableSink` that writes a batch or streaming `Table` to CSV-formatted files. A custom `TableSource` can be defined by implementing the `BatchTableSink` and/or `StreamTableSink` interface. diff --git a/docs/internals/back_pressure_monitoring.md b/docs/internals/back_pressure_monitoring.md index d272eafec59f5..ee25e3b4b00b2 100644 --- a/docs/internals/back_pressure_monitoring.md +++ b/docs/internals/back_pressure_monitoring.md @@ -54,7 +54,7 @@ In order to not overload the task managers with stack trace samples, the web int ## Configuration -You can configure the number of samples for the job manager with the following confiugration keys: +You can configure the number of samples for the job manager with the following configuration keys: - `jobmanager.web.backpressure.refresh-interval`: Time after which available stats are deprecated and need to be refreshed (DEFAULT: 60000, 1 min). - `jobmanager.web.backpressure.num-samples`: Number of stack trace samples to take to determine back pressure (DEFAULT: 100). diff --git a/docs/internals/general_arch.md b/docs/internals/general_arch.md index c599a011d6e35..612fb6b7ff6fc 100644 --- a/docs/internals/general_arch.md +++ b/docs/internals/general_arch.md @@ -83,7 +83,7 @@ You can click on the components in the figure to learn more. ## Projects and Dependencies The Flink system code is divided into multiple sub-projects. The goal is to reduce the number of -dependencies that a project implementing a Flink progam needs, as well as to faciltate easier testing +dependencies that a project implementing a Flink program needs, as well as to faciltate easier testing of smaller sub-modules. The individual projects and their dependencies are shown in the figure below. diff --git a/docs/internals/job_scheduling.md b/docs/internals/job_scheduling.md index cce78d911431a..1e1da97f0d58b 100644 --- a/docs/internals/job_scheduling.md +++ b/docs/internals/job_scheduling.md @@ -60,8 +60,8 @@ and reacts to finished tasks or execution failures. The JobManager receives the {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ "JobGraph" %}, which is a representation of the data flow consisting of operators ({% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertex.java "JobVertex" %}) and intermediate results ({% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java "IntermediateDataSet" %}). -Each operator has properies, like the parallelism and the code that it executes. -In addition, the JobGraph has a set of attached libraries, that are neccessary to execute the code of the operators. +Each operator has properties, like the parallelism and the code that it executes. +In addition, the JobGraph has a set of attached libraries, that are necessary to execute the code of the operators. The JobManager transforms the JobGraph into an {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ "ExecutionGraph" %}. The ExecutionGraph is a parallel version of the JobGraph: For each JobVertex, it contains an {% gh_link /flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java "ExecutionVertex" %} per parallel subtask. An operator with a parallelism of 100 will have one JobVertex and 100 ExecutionVertices. diff --git a/docs/internals/monitoring_rest_api.md b/docs/internals/monitoring_rest_api.md index 97a5c51e10a5e..ca1cf13f7713c 100644 --- a/docs/internals/monitoring_rest_api.md +++ b/docs/internals/monitoring_rest_api.md @@ -39,7 +39,7 @@ The monitoring API is backed by a web server that runs as part of the *JobManage In the case of multiple JobManagers (for high availability), each JobManager will run its own instance of the monitoring API, which offers information about completed and running job while that JobManager was elected the cluster leader. -**NOTE:** Currently, the monitoring API is started together with the new web dashboard. To enable it, one need to add the following entry to the `flink-conf.yaml` in order to activate the new dashboard instread of the old dashboard: `jobmanager.new-web-frontend: true` +**NOTE:** Currently, the monitoring API is started together with the new web dashboard. To enable it, one need to add the following entry to the `flink-conf.yaml` in order to activate the new dashboard instead of the old dashboard: `jobmanager.new-web-frontend: true` ## Developing @@ -169,11 +169,11 @@ Sample Result: **`/joboverview/running`** -Jobs, groupes by status, each with a small summary of its status. The same as `/joboverview`, but containing only currently running jobs. +Jobs, grouped by status, each with a small summary of its status. The same as `/joboverview`, but containing only currently running jobs. **`/joboverview/completed`** -Jobs, groupes by status, each with a small summary of its status. The same as `/joboverview`, but containing only completed (finished, canceled, or failed) jobs. +Jobs, grouped by status, each with a small summary of its status. The same as `/joboverview`, but containing only completed (finished, canceled, or failed) jobs. ### Details of a Running or Completed Job diff --git a/docs/setup/config.md b/docs/setup/config.md index db189a0674aaa..14e9d21a405b6 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -83,7 +83,7 @@ The default fraction for managed memory can be adjusted using the `taskmanager.m ### Memory and Performance Debugging -These options are useful for debugging a Flink application for memory and garbage collection related isues, such as performance and out-of-memory process kills or exceptions. +These options are useful for debugging a Flink application for memory and garbage collection related issues, such as performance and out-of-memory process kills or exceptions. - `taskmanager.debug.memory.startLogThread`: Causes the TaskManagers to periodically log memory and Garbage collection statistics. The statistics include current heap-, off-heap, and other memory pool utilization, as well as the time spent on garbage collection, by heap memory pool. @@ -97,7 +97,7 @@ Flink supports Kerberos authentication of Hadoop services such as HDFS, YARN, or other versions have critical bugs which might fail the Flink job unexpectedly.** -While Hadoop uses Kerberos tickets to authenticate users with services initially, the authentication process continues differently afterwards. Instead of saving the ticket to authenticate on a later access, Hadoop creates its own security tockens (DelegationToken) that it passes around. These are authenticated to Kerberos periodically but are independent of the token renewal time. The tokens have a maximum life span identical to the Kerberos ticket maximum life span. +While Hadoop uses Kerberos tickets to authenticate users with services initially, the authentication process continues differently afterwards. Instead of saving the ticket to authenticate on a later access, Hadoop creates its own security tokens (DelegationToken) that it passes around. These are authenticated to Kerberos periodically but are independent of the token renewal time. The tokens have a maximum life span identical to the Kerberos ticket maximum life span. Please make sure to set the maximum ticket life span high long running jobs. The renewal time of the ticket, on the other hand, is not important because Hadoop abstracts this away using its own security tocken renewal system. Hadoop makes sure that tickets are renewed in time and you can be sure to be authenticated until the end of the ticket life time. @@ -220,7 +220,7 @@ definition. This scheme is used **ONLY** if no other scheme is specified (explic ### Runtime Algorithms -- `taskmanager.runtime.hashjoin-bloom-filters`: Flag to activate/deactivate bloomfilters in the hybrid hash join implementation. In cases where the hash join needs to spill to disk (datasets larger than the reserved fraction of memory), these bloom filters can greatly reduce the number of spilled records, at the cost some CPU cycles. (DEFAULT: false) +- `taskmanager.runtime.hashjoin-bloom-filters`: Flag to activate/deactivate bloom filters in the hybrid hash join implementation. In cases where the hash join needs to spill to disk (datasets larger than the reserved fraction of memory), these bloom filters can greatly reduce the number of spilled records, at the cost some CPU cycles. (DEFAULT: false) - `taskmanager.runtime.max-fan`: The maximal fan-in for external merge joins and fan-out for spilling hash tables. Limits the number of file handles per operator, but may cause intermediate merging/partitioning, if set too small (DEFAULT: 128). - `taskmanager.runtime.sort-spilling-threshold`: A sort operation starts spilling when this fraction of its memory budget is full (DEFAULT: 0.8). @@ -237,7 +237,7 @@ definition. This scheme is used **ONLY** if no other scheme is specified (explic - `yarn.heartbeat-delay` (Default: 5 seconds). Time between heartbeats with the ResourceManager. -- `yarn.properties-file.location` (Default: temp directory). When a Flink job is submitted to YARN, the JobManager's host and the number of available processing slots is written into a properties file, so that the Flink clientis able to pick those details up. This configuration parameter allows changing the default location of that file (for example for environments sharing a Flink installation between users) +- `yarn.properties-file.location` (Default: temp directory). When a Flink job is submitted to YARN, the JobManager's host and the number of available processing slots is written into a properties file, so that the Flink client is able to pick those details up. This configuration parameter allows changing the default location of that file (for example for environments sharing a Flink installation between users) - `yarn.application-master.env.`*ENV_VAR1=value* Configuration values prefixed with `yarn.application-master.env.` will be passed as environment variables to the ApplicationMaster/JobManager process. For example for passing `LD_LIBRARY_PATH` as an env variable to the ApplicationMaster, set: diff --git a/docs/setup/local_setup.md b/docs/setup/local_setup.md index 54864f065c118..43a9b791c247c 100644 --- a/docs/setup/local_setup.md +++ b/docs/setup/local_setup.md @@ -102,7 +102,7 @@ Note: The ``bin`` folder of your Java Runtime Environment must be included in Wi $ cd flink $ cd bin $ start-local.bat -Starting Flink job manager. Webinterface by default on http://localhost:8081/. +Starting Flink job manager. Web interface by default on http://localhost:8081/. Do not close this batch window. Stop job manager by pressing Ctrl+C. ~~~ diff --git a/docs/setup/yarn_setup.md b/docs/setup/yarn_setup.md index f24f451361703..aa0f7a4bdc9b7 100644 --- a/docs/setup/yarn_setup.md +++ b/docs/setup/yarn_setup.md @@ -221,7 +221,7 @@ Note: It is possible to combine `-m yarn-cluster` with a detached YARN submissio Flink's YARN client has the following configuration parameters to control how to behave in case of container failures. These parameters can be set either from the `conf/flink-conf.yaml` or when starting the YARN session, using `-D` parameters. - `yarn.reallocate-failed`: This parameter controls whether Flink should reallocate failed TaskManager containers. Default: true -- `yarn.maximum-failed-containers`: The maximum number of failed containers the ApplicationMaster accepts until it fails the YARN session. Default: The number of initally requested TaskManagers (`-n`). +- `yarn.maximum-failed-containers`: The maximum number of failed containers the ApplicationMaster accepts until it fails the YARN session. Default: The number of initially requested TaskManagers (`-n`). - `yarn.application-attempts`: The number of ApplicationMaster (+ its TaskManager containers) attempts. If this value is set to 1 (default), the entire YARN session will fail when the Application master fails. Higher values specify the number of restarts of the ApplicationMaster by YARN. ## Debugging a failed YARN session @@ -240,11 +240,11 @@ yarn logs -applicationId Note that it takes a few seconds after the session has finished until the logs show up. -### YARN Client console & Webinterfaces +### YARN Client console & Web interfaces The Flink YARN client also prints error messages in the terminal if errors occur during runtime (for example if a TaskManager stops working after some time). -In addition to that, there is the YARN Resource Manager webinterface (by default on port 8088). The port of the Resource Manager web interface is determined by the `yarn.resourcemanager.webapp.address` configuration value. +In addition to that, there is the YARN Resource Manager web interface (by default on port 8088). The port of the Resource Manager web interface is determined by the `yarn.resourcemanager.webapp.address` configuration value. It allows to access log files for running YARN applications and shows diagnostics for failed apps. @@ -261,7 +261,7 @@ ranges configured, users can also submit jobs to Flink crossing the firewall. Currently, two services are needed to submit a job: - * The JobManager (ApplicatonMaster in YARN) + * The JobManager (ApplicationMaster in YARN) * The BlobServer running within the JobManager. When submitting a job to Flink, the BlobServer will distribute the jars with the user code to all worker nodes (TaskManagers). From 34f9db1fa028e0700a61c2b37e7bd24ba93c5b92 Mon Sep 17 00:00:00 2001 From: Joshi Date: Mon, 16 May 2016 14:25:12 -0700 Subject: [PATCH 27/70] [FLINK-3900] [tableAPI] Set nullCheck=true as default in TableConfig. This closes #1994 --- docs/apis/table.md | 2 +- .../src/main/scala/org/apache/flink/api/table/TableConfig.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/apis/table.md b/docs/apis/table.md index eae4ebd88bd34..276341db154df 100644 --- a/docs/apis/table.md +++ b/docs/apis/table.md @@ -737,5 +737,5 @@ Runtime Configuration The Table API provides a configuration (the so-called `TableConfig`) to modify runtime behavior. It can be accessed either through `TableEnvironment` or passed to the `toDataSet`/`toDataStream` method when using Scala implicit conversion. ### Null Handling -By default, the Table API does not support `null` values at runtime for efficiency purposes. Null handling can be enabled by setting the `nullCheck` property in the `TableConfig` to `true`. +By default, the Table API supports `null` values. Null handling can be disabled by setting the `nullCheck` property in the `TableConfig` to `false`. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala index e93d37d83d6ca..c92451d3a9bcd 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala @@ -32,7 +32,7 @@ class TableConfig extends Serializable { /** * Defines if all fields need to be checked for NULL first. */ - private var nullCheck: Boolean = false + private var nullCheck: Boolean = true /** * Defines if efficient types (such as Tuple types or Atomic types) From 9a74e8993cd3ed2eded2847f3a560c16e5c06b77 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Wed, 13 Apr 2016 16:46:58 +0800 Subject: [PATCH 28/70] [FLINK-3754] [tableAPI] Add validation phase to Table API before construction of RelNodes. This closes #1958 --- .../flink/api/scala/table/expressionDsl.scala | 74 ++-- .../api/table/BatchTableEnvironment.scala | 34 +- .../flink/api/table/FlinkPlannerImpl.scala | 4 +- .../api/table/StreamTableEnvironment.scala | 17 +- .../flink/api/table/TableEnvironment.scala | 43 ++- ...ParserException.scala => exceptions.scala} | 22 +- .../api/table/expressions/Expression.scala | 64 +++- .../table/expressions/ExpressionParser.scala | 42 +-- .../api/table/expressions/InputTypeSpec.scala | 55 +++ .../api/table/expressions/TreeNode.scala | 120 ------- .../api/table/expressions/aggregations.scala | 33 +- .../api/table/expressions/arithmetic.scala | 44 ++- .../flink/api/table/expressions/call.scala | 75 +--- .../flink/api/table/expressions/cast.scala | 21 +- .../api/table/expressions/comparison.scala | 41 ++- .../table/expressions/fieldExpression.scala | 73 +++- .../api/table/expressions/literals.scala | 14 +- .../flink/api/table/expressions/logic.scala | 50 ++- .../table/expressions/mathExpressions.scala | 115 ++++++ .../api/table/expressions/ordering.scala | 21 +- .../table/expressions/stringExpressions.scala | 220 ++++++++++++ .../api/table/plan/RexNodeTranslator.scala | 50 ++- .../api/table/plan/logical/LogicalNode.scala | 162 +++++++++ .../api/table/plan/logical/operators.scala | 339 ++++++++++++++++++ .../org/apache/flink/api/table/table.scala | 296 +++------------ .../flink/api/table/trees/TreeNode.scala | 114 ++++++ .../api/table/typeutils/TypeCheckUtils.scala | 40 +++ .../api/table/typeutils/TypeCoercion.scala | 92 +++++ .../ExprValidationResult.scala} | 26 +- .../api/table/validate/FunctionCatalog.scala | 124 +++++++ .../java/batch/TableEnvironmentITCase.java | 11 +- .../java/batch/table/AggregationsITCase.java | 8 +- .../java/batch/table/ExpressionsITCase.java | 4 +- .../api/java/batch/table/FilterITCase.java | 3 +- .../table/GroupedAggregationsITCase.java | 5 +- .../api/java/batch/table/JoinITCase.java | 10 +- .../api/java/batch/table/SelectITCase.java | 5 +- .../batch/table/StringExpressionsITCase.java | 32 +- .../api/java/batch/table/UnionITCase.java | 10 +- .../scala/batch/TableEnvironmentITCase.scala | 6 +- .../batch/table/AggregationsITCase.scala | 9 +- .../scala/batch/table/ExpressionsITCase.scala | 4 +- .../api/scala/batch/table/FilterITCase.scala | 4 +- .../table/GroupedAggregationsITCase.scala | 6 +- .../api/scala/batch/table/JoinITCase.scala | 10 +- .../api/scala/batch/table/SelectITCase.scala | 8 +- .../batch/table/StringExpressionsITCase.scala | 7 +- .../api/scala/batch/table/UnionITCase.scala | 8 +- .../utils/ExpressionEvaluator.scala | 10 +- .../api/scala/stream/table/UnionITCase.scala | 10 +- .../stream/table/UnsupportedOpsTest.scala | 7 + 51 files changed, 1876 insertions(+), 726 deletions(-) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/{ExpressionParserException.scala => exceptions.scala} (59%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/TreeNode.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/{TableException.scala => validate/ExprValidationResult.scala} (56%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala index 0f0b93ce82b7b..11fb64a40b4eb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala @@ -17,11 +17,11 @@ */ package org.apache.flink.api.scala.table +import scala.language.implicitConversions + import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.table.expressions._ -import scala.language.implicitConversions - /** * These are all the operations that can be used to construct an [[Expression]] AST for expression * operations. @@ -63,7 +63,7 @@ trait ImplicitExpressionOperations { def cast(toType: TypeInformation[_]) = Cast(expr, toType) - def as(name: Symbol) = Naming(expr, name.name) + def as(name: Symbol) = Alias(expr, name.name) def asc = Asc(expr) def desc = Desc(expr) @@ -91,37 +91,37 @@ trait ImplicitExpressionOperations { /** * Calculates the Euler's number raised to the given power. */ - def exp() = Call(BuiltInFunctionNames.EXP, expr) + def exp() = Exp(expr) /** * Calculates the base 10 logarithm of given value. */ - def log10() = Call(BuiltInFunctionNames.LOG10, expr) + def log10() = Log10(expr) /** * Calculates the natural logarithm of given value. */ - def ln() = Call(BuiltInFunctionNames.LN, expr) + def ln() = Ln(expr) /** * Calculates the given number raised to the power of the other value. */ - def power(other: Expression) = Call(BuiltInFunctionNames.POWER, expr, other) + def power(other: Expression) = Power(expr, other) /** * Calculates the absolute value of given one. */ - def abs() = Call(BuiltInFunctionNames.ABS, expr) + def abs() = Abs(expr) /** * Calculates the largest integer less than or equal to a given number. */ - def floor() = Call(BuiltInFunctionNames.FLOOR, expr) + def floor() = Floor(expr) /** * Calculates the smallest integer greater than or equal to a given number. */ - def ceil() = Call(BuiltInFunctionNames.CEIL, expr) + def ceil() = Ceil(expr) /** * Creates a substring of the given string between the given indices. @@ -130,9 +130,8 @@ trait ImplicitExpressionOperations { * @param endIndex last character of the substring (starting at 1, inclusive) * @return substring */ - def substring(beginIndex: Expression, endIndex: Expression) = { - Call(BuiltInFunctionNames.SUBSTRING, expr, beginIndex, endIndex) - } + def substring(beginIndex: Expression, endIndex: Expression) = + SubString(expr, beginIndex, endIndex) /** * Creates a substring of the given string beginning at the given index to the end. @@ -140,9 +139,8 @@ trait ImplicitExpressionOperations { * @param beginIndex first character of the substring (starting at 1, inclusive) * @return substring */ - def substring(beginIndex: Expression) = { - Call(BuiltInFunctionNames.SUBSTRING, expr, beginIndex) - } + def substring(beginIndex: Expression) = + new SubString(expr, beginIndex) /** * Removes leading and/or trailing characters from the given string. @@ -155,25 +153,13 @@ trait ImplicitExpressionOperations { def trim( removeLeading: Boolean = true, removeTrailing: Boolean = true, - character: Expression = BuiltInFunctionConstants.TRIM_DEFAULT_CHAR) = { + character: Expression = TrimConstants.TRIM_DEFAULT_CHAR) = { if (removeLeading && removeTrailing) { - Call( - BuiltInFunctionNames.TRIM, - BuiltInFunctionConstants.TRIM_BOTH, - character, - expr) + Trim(TrimConstants.TRIM_BOTH, character, expr) } else if (removeLeading) { - Call( - BuiltInFunctionNames.TRIM, - BuiltInFunctionConstants.TRIM_LEADING, - character, - expr) + Trim(TrimConstants.TRIM_LEADING, character, expr) } else if (removeTrailing) { - Call( - BuiltInFunctionNames.TRIM, - BuiltInFunctionConstants.TRIM_TRAILING, - character, - expr) + Trim(TrimConstants.TRIM_TRAILING, character, expr) } else { expr } @@ -182,51 +168,39 @@ trait ImplicitExpressionOperations { /** * Returns the length of a String. */ - def charLength() = { - Call(BuiltInFunctionNames.CHAR_LENGTH, expr) - } + def charLength() = CharLength(expr) /** * Returns all of the characters in a String in upper case using the rules of * the default locale. */ - def upperCase() = { - Call(BuiltInFunctionNames.UPPER_CASE, expr) - } + def upperCase() = Upper(expr) /** * Returns all of the characters in a String in lower case using the rules of * the default locale. */ - def lowerCase() = { - Call(BuiltInFunctionNames.LOWER_CASE, expr) - } + def lowerCase() = Lower(expr) /** * Converts the initial letter of each word in a String to uppercase. * Assumes a String containing only [A-Za-z0-9], everything else is treated as whitespace. */ - def initCap() = { - Call(BuiltInFunctionNames.INIT_CAP, expr) - } + def initCap() = InitCap(expr) /** * Returns true, if a String matches the specified LIKE pattern. * * e.g. "Jo_n%" matches all Strings that start with "Jo(arbitrary letter)n" */ - def like(pattern: Expression) = { - Call(BuiltInFunctionNames.LIKE, expr, pattern) - } + def like(pattern: Expression) = Like(expr, pattern) /** * Returns true, if a String matches the specified SQL regex pattern. * * e.g. "A+" matches all Strings that consist of at least one A */ - def similar(pattern: Expression) = { - Call(BuiltInFunctionNames.SIMILAR, expr, pattern) - } + def similar(pattern: Expression) = Similar(expr, pattern) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala index 39e3105111f5e..207500a723472 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala @@ -24,6 +24,7 @@ import org.apache.calcite.plan.RelOptPlanner.CannotPlanException import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.sql2rel.RelDecorrelator import org.apache.calcite.tools.Programs + import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.{ExecutionEnvironment, DataSet} import org.apache.flink.api.java.io.DiscardingOutputFormat @@ -31,7 +32,8 @@ import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.api.table.explain.PlanJsonParser import org.apache.flink.api.table.expressions.Expression import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.plan.nodes.dataset.{DataSetRel, DataSetConvention} +import org.apache.flink.api.table.plan.logical.{CatalogNode, LogicalRelNode} +import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetRel} import org.apache.flink.api.table.plan.rules.FlinkRuleSets import org.apache.flink.api.table.plan.schema.{TableSourceTable, DataSetTable} import org.apache.flink.api.table.sinks.{BatchTableSink, TableSink} @@ -72,7 +74,7 @@ abstract class BatchTableEnvironment( val m = internalNamePattern.findFirstIn(name) m match { case Some(_) => - throw new TableException(s"Illegal Table name. " + + throw new ValidationException(s"Illegal Table name. " + s"Please choose a name that does not contain the pattern $internalNamePattern") case None => } @@ -87,18 +89,15 @@ abstract class BatchTableEnvironment( * The table to scan must be registered in the [[TableEnvironment]]'s catalog. * * @param tableName The name of the table to scan. - * @throws TableException if no table is registered under the given name. + * @throws ValidationException if no table is registered under the given name. * @return The scanned table. */ - @throws[TableException] + @throws[ValidationException] def scan(tableName: String): Table = { - if (isRegistered(tableName)) { - relBuilder.scan(tableName) - new Table(relBuilder.build(), this) - } - else { - throw new TableException(s"Table \'$tableName\' was not found in the registry.") + new Table(this, CatalogNode(tableName, getRowType(tableName))) + } else { + throw new ValidationException(s"Table \'$tableName\' was not found in the registry.") } } @@ -133,7 +132,7 @@ abstract class BatchTableEnvironment( // transform to a relational tree val relational = planner.rel(validated) - new Table(relational.rel, this) + new Table(this, LogicalRelNode(relational.rel)) } /** @@ -169,7 +168,7 @@ abstract class BatchTableEnvironment( */ private[flink] def explain(table: Table, extended: Boolean): String = { - val ast = RelOptUtil.toString(table.relNode) + val ast = RelOptUtil.toString(table.getRelNode) val dataSet = translate[Row](table)(TypeExtractor.createTypeInfo(classOf[Row])) dataSet.output(new DiscardingOutputFormat[Row]) val env = dataSet.getExecutionEnvironment @@ -219,15 +218,10 @@ abstract class BatchTableEnvironment( * @tparam T The type of the [[DataSet]]. */ protected def registerDataSetInternal[T]( - name: String, dataSet: DataSet[T], - fields: Array[Expression]): Unit = { + name: String, dataSet: DataSet[T], fields: Array[Expression]): Unit = { - val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType, fields.toArray) - val dataSetTable = new DataSetTable[T]( - dataSet, - fieldIndexes.toArray, - fieldNames.toArray - ) + val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType, fields) + val dataSetTable = new DataSetTable[T](dataSet, fieldIndexes, fieldNames) registerTableInternal(name, dataSetTable) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala index 5a1b3fe8cb5ef..9d0a146ef1dc7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala @@ -34,7 +34,7 @@ import org.apache.calcite.sql.parser.{SqlParser, SqlParseException} import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql.{SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlToRelConverter, SqlRexConvertletTable} -import org.apache.calcite.tools.{RelConversionException, ValidationException, Frameworks, FrameworkConfig} +import org.apache.calcite.tools.{RelConversionException, ValidationException => CValidationException, Frameworks, FrameworkConfig} import org.apache.calcite.util.Util import scala.collection.JavaConversions._ @@ -96,7 +96,7 @@ class FlinkPlannerImpl(config: FrameworkConfig, var planner: RelOptPlanner) { } catch { case e: RuntimeException => { - throw new ValidationException(e) + throw new CValidationException(e) } } validatedSqlNode diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala index be1c00569c4e4..8ba30002f72e3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala @@ -24,10 +24,12 @@ import org.apache.calcite.plan.RelOptPlanner.CannotPlanException import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.sql2rel.RelDecorrelator import org.apache.calcite.tools.Programs + import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.table.expressions.Expression import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamRel, DataStreamConvention} +import org.apache.flink.api.table.plan.logical.{CatalogNode, LogicalRelNode} +import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamConvention, DataStreamRel} import org.apache.flink.api.table.plan.rules.FlinkRuleSets import org.apache.flink.api.table.sinks.{StreamTableSink, TableSink} import org.apache.flink.api.table.plan.schema. @@ -86,18 +88,17 @@ abstract class StreamTableEnvironment( * The table to ingest must be registered in the [[TableEnvironment]]'s catalog. * * @param tableName The name of the table to ingest. - * @throws TableException if no table is registered under the given name. + * @throws ValidationException if no table is registered under the given name. * @return The ingested table. */ - @throws[TableException] + @throws[ValidationException] def ingest(tableName: String): Table = { if (isRegistered(tableName)) { - relBuilder.scan(tableName) - new Table(relBuilder.build(), this) + new Table(this, CatalogNode(tableName, getRowType(tableName))) } else { - throw new TableException(s"Table \'$tableName\' was not found in the registry.") + throw new ValidationException(s"Table \'$tableName\' was not found in the registry.") } } @@ -132,7 +133,7 @@ abstract class StreamTableEnvironment( // transform to a relational tree val relational = planner.rel(validated) - new Table(relational.rel, this) + new Table(this, LogicalRelNode(relational.rel)) } /** @@ -240,7 +241,7 @@ abstract class StreamTableEnvironment( */ protected def translate[A](table: Table)(implicit tpe: TypeInformation[A]): DataStream[A] = { - val relNode = table.relNode + val relNode = table.getRelNode // decorrelate val decorPlan = RelDecorrelator.decorrelateQuery(relNode) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala index 6ccde47feee7a..8aa9e1063f8e2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala @@ -21,11 +21,13 @@ package org.apache.flink.api.table import java.util.concurrent.atomic.AtomicInteger import org.apache.calcite.config.Lex -import org.apache.calcite.plan.RelOptPlanner +import org.apache.calcite.plan.{RelOptCluster, RelOptPlanner} +import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.calcite.schema.SchemaPlus import org.apache.calcite.schema.impl.AbstractTable import org.apache.calcite.sql.parser.SqlParser -import org.apache.calcite.tools.{Frameworks, FrameworkConfig, RelBuilder} +import org.apache.calcite.tools.{FrameworkConfig, Frameworks, RelBuilder} + import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} import org.apache.flink.api.java.{ExecutionEnvironment => JavaBatchExecEnv} import org.apache.flink.api.java.table.{BatchTableEnvironment => JavaBatchTableEnv} @@ -35,11 +37,11 @@ import org.apache.flink.api.scala.{ExecutionEnvironment => ScalaBatchExecEnv} import org.apache.flink.api.scala.table.{BatchTableEnvironment => ScalaBatchTableEnv} import org.apache.flink.api.scala.table.{StreamTableEnvironment => ScalaStreamTableEnv} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo -import org.apache.flink.api.table.expressions.{Naming, UnresolvedFieldReference, Expression} +import org.apache.flink.api.table.expressions.{Alias, Expression, UnresolvedFieldReference} import org.apache.flink.api.table.plan.cost.DataSetCostFactory -import org.apache.flink.api.table.plan.schema.{TransStreamTable, RelTable} import org.apache.flink.api.table.sinks.TableSink -import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.api.table.plan.schema.{RelTable, TransStreamTable} +import org.apache.flink.api.table.validate.FunctionCatalog import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaStreamExecEnv} import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv} @@ -72,10 +74,16 @@ abstract class TableEnvironment(val config: TableConfig) { // the builder for Calcite RelNodes, Calcite's representation of a relational expression tree. protected val relBuilder: RelBuilder = RelBuilder.create(frameworkConfig) - // the planner instance used to optimize queries of this TableEnvironment - private val planner: RelOptPlanner = relBuilder + private val cluster: RelOptCluster = relBuilder .values(Array("dummy"), new Integer(1)) - .build().getCluster.getPlanner + .build().getCluster + + // the planner instance used to optimize queries of this TableEnvironment + private val planner: RelOptPlanner = cluster.getPlanner + + private val typeFactory: RelDataTypeFactory = cluster.getTypeFactory + + private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuildIns // a counter for unique attribute names private val attrNameCntr: AtomicInteger = new AtomicInteger(0) @@ -94,7 +102,7 @@ abstract class TableEnvironment(val config: TableConfig) { // check that table belongs to this table environment if (table.tableEnv != this) { - throw new TableException( + throw new ValidationException( "Only tables that belong to this TableEnvironment can be registered.") } @@ -152,7 +160,7 @@ abstract class TableEnvironment(val config: TableConfig) { * * @param name The name under which the table is registered. * @param table The table to register in the catalog - * @throws TableException if another table is registered under the provided name. + * @throws ValidationException if another table is registered under the provided name. */ @throws[TableException] protected def registerTableInternal(name: String, table: AbstractTable): Unit = { @@ -182,6 +190,10 @@ abstract class TableEnvironment(val config: TableConfig) { tables.getTableNames.contains(name) } + protected def getRowType(name: String): RelDataType = { + tables.getTable(name).getRowType(typeFactory) + } + /** Returns a unique temporary attribute name. */ private[flink] def createUniqueAttributeName(): String = { "TMP_" + attrNameCntr.getAndIncrement() @@ -197,6 +209,10 @@ abstract class TableEnvironment(val config: TableConfig) { planner } + private[flink] def getFunctionCatalog: FunctionCatalog = { + functionCatalog + } + /** Returns the Calcite [[FrameworkConfig]] of this TableEnvironment. */ private[flink] def getFrameworkConfig: FrameworkConfig = { frameworkConfig @@ -253,7 +269,7 @@ abstract class TableEnvironment(val config: TableConfig) { case t: TupleTypeInfo[A] => exprs.zipWithIndex.map { case (UnresolvedFieldReference(name), idx) => (idx, name) - case (Naming(UnresolvedFieldReference(origName), name), _) => + case (Alias(UnresolvedFieldReference(origName), name), _) => val idx = t.getFieldIndex(origName) if (idx < 0) { throw new IllegalArgumentException(s"$origName is not a field of type $t") @@ -265,7 +281,7 @@ abstract class TableEnvironment(val config: TableConfig) { case c: CaseClassTypeInfo[A] => exprs.zipWithIndex.map { case (UnresolvedFieldReference(name), idx) => (idx, name) - case (Naming(UnresolvedFieldReference(origName), name), _) => + case (Alias(UnresolvedFieldReference(origName), name), _) => val idx = c.getFieldIndex(origName) if (idx < 0) { throw new IllegalArgumentException(s"$origName is not a field of type $c") @@ -276,7 +292,7 @@ abstract class TableEnvironment(val config: TableConfig) { } case p: PojoTypeInfo[A] => exprs.map { - case Naming(UnresolvedFieldReference(origName), name) => + case Alias(UnresolvedFieldReference(origName), name) => val idx = p.getFieldIndex(origName) if (idx < 0) { throw new IllegalArgumentException(s"$origName is not a field of type $p") @@ -389,5 +405,4 @@ object TableEnvironment { new ScalaStreamTableEnv(executionEnvironment, tableConfig) } - } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/ExpressionParserException.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala similarity index 59% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/ExpressionParserException.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala index 2d6fae6029fe0..a3ab6fdda0b7c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/ExpressionParserException.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala @@ -15,9 +15,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.api.table /** - * Exception for all errors occurring during expression evaluation. - */ -class ExpressionParserException(msg: String) extends RuntimeException(msg) + * Exception for all errors occurring during expression parsing. + */ +case class ExpressionParserException(msg: String) extends RuntimeException(msg) + +/** + * General Exception for all errors during table handling. + */ +case class TableException(msg: String) extends RuntimeException(msg) + +/** + * Exception for all errors occurring during validation phase. + */ +case class ValidationException(msg: String) extends RuntimeException(msg) + +/** + * Exception for unwanted method calling on unresolved expression. + */ +case class UnresolvedException(msg: String) extends RuntimeException(msg) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala index 6960a9f14b300..14e464e9d6aa0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala @@ -17,13 +17,34 @@ */ package org.apache.flink.api.table.expressions -import java.util.concurrent.atomic.AtomicInteger - import org.apache.calcite.rex.RexNode import org.apache.calcite.tools.RelBuilder -abstract class Expression extends TreeNode[Expression] { self: Product => - def name: String = Expression.freshName("expression") +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.trees.TreeNode +import org.apache.flink.api.table.validate.{ExprValidationResult, ValidationSuccess} + +abstract class Expression extends TreeNode[Expression] { + /** + * Returns the [[TypeInformation]] for evaluating this expression. + * It is sometimes not available until the expression is valid. + */ + def resultType: TypeInformation[_] + + /** + * One pass validation of the expression tree in post order. + */ + lazy val valid: Boolean = childrenValid && validateInput().isSuccess + + def childrenValid: Boolean = children.forall(_.valid) + + /** + * Check input data types, inputs number or other properties specified by this expression. + * Return `ValidationSuccess` if it pass the check, + * or `ValidationFailure` with supplement message explaining the error. + * Note: we should only call this method until `childrenValid == true` + */ + def validateInput(): ExprValidationResult = ValidationSuccess /** * Convert Expression to its counterpart in Calcite, i.e. RexNode @@ -32,31 +53,36 @@ abstract class Expression extends TreeNode[Expression] { self: Product => throw new UnsupportedOperationException( s"${this.getClass.getName} cannot be transformed to RexNode" ) + + def checkEquals(other: Expression): Boolean = { + if (this.getClass != other.getClass) { + false + } else { + def checkEquality(elements1: Seq[Any], elements2: Seq[Any]): Boolean = { + elements1.length == elements2.length && elements1.zip(elements2).forall { + case (e1: Expression, e2: Expression) => e1.checkEquals(e2) + case (t1: Seq[_], t2: Seq[_]) => checkEquality(t1, t2) + case (i1, i2) => i1 == i2 + } + } + val elements1 = this.productIterator.toSeq + val elements2 = other.productIterator.toSeq + checkEquality(elements1, elements2) + } + } } -abstract class BinaryExpression extends Expression { self: Product => +abstract class BinaryExpression extends Expression { def left: Expression def right: Expression def children = Seq(left, right) } -abstract class UnaryExpression extends Expression { self: Product => +abstract class UnaryExpression extends Expression { def child: Expression def children = Seq(child) } -abstract class LeafExpression extends Expression { self: Product => +abstract class LeafExpression extends Expression { val children = Nil } - -case class NopExpression() extends LeafExpression { - override val name = Expression.freshName("nop") -} - -object Expression { - def freshName(prefix: String): String = { - s"$prefix-${freshNameCounter.getAndIncrement}" - } - - val freshNameCounter = new AtomicInteger -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala index ffadca506a908..db3d18764a25d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala @@ -111,8 +111,8 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { stringLiteralFlink | singleQuoteStringLiteral | boolLiteral | nullLiteral - lazy val fieldReference: PackratParser[Expression] = ident ^^ { - case sym => UnresolvedFieldReference(sym) + lazy val fieldReference: PackratParser[NamedExpression] = ident ^^ { + sym => UnresolvedFieldReference(sym) } lazy val atom: PackratParser[Expression] = @@ -155,7 +155,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val suffixAs: PackratParser[Expression] = composite ~ "." ~ AS ~ "(" ~ fieldReference ~ ")" ^^ { - case e ~ _ ~ _ ~ _ ~ target ~ _ => Naming(e, target.name) + case e ~ _ ~ _ ~ _ ~ target ~ _ => Alias(e, target.name) } lazy val suffixEval: PackratParser[Expression] = @@ -165,27 +165,23 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val suffixFunctionCall = composite ~ "." ~ functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ { - case operand ~ _ ~ name ~ _ ~ args ~ _ => Call(name.toUpperCase, operand :: args : _*) + case operand ~ _ ~ name ~ _ ~ args ~ _ => Call(name.toUpperCase, operand :: args) } lazy val suffixTrim = composite ~ ".trim(" ~ ("BOTH" | "LEADING" | "TRAILING") ~ "," ~ expression ~ ")" ^^ { case operand ~ _ ~ trimType ~ _ ~ trimCharacter ~ _ => val flag = trimType match { - case "BOTH" => BuiltInFunctionConstants.TRIM_BOTH - case "LEADING" => BuiltInFunctionConstants.TRIM_LEADING - case "TRAILING" => BuiltInFunctionConstants.TRIM_TRAILING + case "BOTH" => TrimConstants.TRIM_BOTH + case "LEADING" => TrimConstants.TRIM_LEADING + case "TRAILING" => TrimConstants.TRIM_TRAILING } - Call(BuiltInFunctionNames.TRIM, flag, trimCharacter, operand) + Trim(flag, trimCharacter, operand) } lazy val suffixTrimWithoutArgs = composite <~ ".trim" ~ opt("()") ^^ { case e => - Call( - BuiltInFunctionNames.TRIM, - BuiltInFunctionConstants.TRIM_BOTH, - BuiltInFunctionConstants.TRIM_DEFAULT_CHAR, - e) + Trim(TrimConstants.TRIM_BOTH, TrimConstants.TRIM_DEFAULT_CHAR, e) } lazy val suffixed: PackratParser[Expression] = @@ -223,7 +219,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val prefixAs: PackratParser[Expression] = AS ~ "(" ~ expression ~ "," ~ fieldReference ~ ")" ^^ { - case _ ~ _ ~ e ~ _ ~ target ~ _ => Naming(e, target.name) + case _ ~ _ ~ e ~ _ ~ target ~ _ => Alias(e, target.name) } lazy val prefixEval: PackratParser[Expression] = composite ~ @@ -232,27 +228,23 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { } lazy val prefixFunctionCall = functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ { - case name ~ _ ~ args ~ _ => Call(name.toUpperCase, args: _*) + case name ~ _ ~ args ~ _ => Call(name.toUpperCase, args) } lazy val prefixTrim = "trim(" ~ ("BOTH" | "LEADING" | "TRAILING") ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ { case _ ~ trimType ~ _ ~ trimCharacter ~ _ ~ operand ~ _ => val flag = trimType match { - case "BOTH" => BuiltInFunctionConstants.TRIM_BOTH - case "LEADING" => BuiltInFunctionConstants.TRIM_LEADING - case "TRAILING" => BuiltInFunctionConstants.TRIM_TRAILING + case "BOTH" => TrimConstants.TRIM_BOTH + case "LEADING" => TrimConstants.TRIM_LEADING + case "TRAILING" => TrimConstants.TRIM_TRAILING } - Call(BuiltInFunctionNames.TRIM, flag, trimCharacter, operand) + Trim(flag, trimCharacter, operand) } lazy val prefixTrimWithoutArgs = "trim(" ~ expression ~ ")" ^^ { case _ ~ operand ~ _ => - Call( - BuiltInFunctionNames.TRIM, - BuiltInFunctionConstants.TRIM_BOTH, - BuiltInFunctionConstants.TRIM_DEFAULT_CHAR, - operand) + Trim(TrimConstants.TRIM_BOTH, TrimConstants.TRIM_DEFAULT_CHAR, operand) } lazy val prefixed: PackratParser[Expression] = @@ -322,7 +314,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { // alias lazy val alias: PackratParser[Expression] = logic ~ AS ~ fieldReference ^^ { - case e ~ _ ~ name => Naming(e, name.name) + case e ~ _ ~ name => Alias(e, name.name) } | logic lazy val expression: PackratParser[Expression] = alias diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala new file mode 100644 index 0000000000000..9cb52d59510be --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.expressions + +import scala.collection.mutable + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.validate._ + +/** + * Expressions that have specification on its inputs. + */ +trait InputTypeSpec extends Expression { + + /** + * Input type specification for each child. + * + * For example, [[Power]] expecting both of the children be of Double Type should use: + * {{{ + * def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: DOUBLE_TYPE_INFO :: Nil + * }}} + */ + def expectedTypes: Seq[TypeInformation[_]] + + override def validateInput(): ExprValidationResult = { + val typeMismatches = mutable.ArrayBuffer.empty[String] + children.zip(expectedTypes).zipWithIndex.foreach { case ((e, tpe), i) => + if (e.resultType != tpe) { + typeMismatches += s"expecting $tpe on ${i}th input, get ${e.resultType}" + } + } + if (typeMismatches.isEmpty) { + ValidationSuccess + } else { + ValidationFailure( + s"$this fails on input type checking: ${typeMismatches.mkString("[", ", ", "]")}") + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/TreeNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/TreeNode.scala deleted file mode 100644 index 9d4ca800955c2..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/TreeNode.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flink.api.table.expressions - -/** - * Generic base class for trees that can be transformed and traversed. - */ -abstract class TreeNode[A <: TreeNode[A]] { self: A with Product => - - /** - * List of child nodes that should be considered when doing transformations. Other values - * in the Product will not be transformed, only handed through. - */ - def children: Seq[A] - - /** - * Tests for equality by first testing for reference equality. - */ - def fastEquals(other: TreeNode[_]): Boolean = this.eq(other) || this == other - - def transformPre(rule: PartialFunction[A, A]): A = { - val afterTransform = rule.applyOrElse(this, identity[A]) - - if (afterTransform fastEquals this) { - this.transformChildrenPre(rule) - } else { - afterTransform.transformChildrenPre(rule) - } - } - - def transformChildrenPre(rule: PartialFunction[A, A]): A = { - var changed = false - val newArgs = productIterator map { - case child: A if children.contains(child) => - val newChild = child.transformPre(rule) - if (newChild fastEquals child) { - child - } else { - changed = true - newChild - } - case other: AnyRef => other - case null => null - } toArray - - if (changed) makeCopy(newArgs) else this - } - - def transformPost(rule: PartialFunction[A, A]): A = { - val afterChildren = transformChildrenPost(rule) - if (afterChildren fastEquals this) { - rule.applyOrElse(this, identity[A]) - } else { - rule.applyOrElse(afterChildren, identity[A]) - } - } - - def transformChildrenPost(rule: PartialFunction[A, A]): A = { - var changed = false - val newArgs = productIterator map { - case child: A if children.contains(child) => - val newChild = child.transformPost(rule) - if (newChild fastEquals child) { - child - } else { - changed = true - newChild - } - case other: AnyRef => other - case null => null - } toArray - // toArray forces evaluation, toSeq does not seem to work here - - if (changed) makeCopy(newArgs) else this - } - - def exists(predicate: A => Boolean): Boolean = { - var exists = false - this.transformPre { - case e: A => if (predicate(e)) { - exists = true - } - e - } - exists - } - - /** - * Creates a new copy of this expression with new children. This is used during transformation - * if children change. This must be overridden by tree nodes that don't have the Constructor - * arguments in the same order as the `children`. - */ - def makeCopy(newArgs: Seq[AnyRef]): this.type = { - val defaultCtor = - this.getClass.getConstructors.find { _.getParameterTypes.size > 0}.head - try { - defaultCtor.newInstance(newArgs.toArray: _*).asInstanceOf[this.type] - } catch { - case iae: IllegalArgumentException => - println("IAE " + this) - throw new RuntimeException("Should never happen.") - } - } -} - diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala index 8cd9dc3873ca4..24ce85f159b52 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala @@ -22,7 +22,10 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.tools.RelBuilder import org.apache.calcite.tools.RelBuilder.AggCall -abstract sealed class Aggregation extends UnaryExpression { self: Product => +import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.api.table.typeutils.TypeCheckUtils + +abstract sealed class Aggregation extends UnaryExpression { override def toString = s"Aggregate($child)" @@ -36,41 +39,59 @@ abstract sealed class Aggregation extends UnaryExpression { self: Product => } case class Sum(child: Expression) extends Aggregation { - override def toString = s"($child).sum" + override def toString = s"sum($child)" override def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = { relBuilder.aggregateCall(SqlStdOperatorTable.SUM, false, null, name, child.toRexNode) } + + override def resultType = child.resultType + + override def validateInput = TypeCheckUtils.assertNumericExpr(child.resultType, "sum") } case class Min(child: Expression) extends Aggregation { - override def toString = s"($child).min" + override def toString = s"min($child)" override def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = { relBuilder.aggregateCall(SqlStdOperatorTable.MIN, false, null, name, child.toRexNode) } + + override def resultType = child.resultType + + override def validateInput = TypeCheckUtils.assertOrderableExpr(child.resultType, "min") } case class Max(child: Expression) extends Aggregation { - override def toString = s"($child).max" + override def toString = s"max($child)" override def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = { relBuilder.aggregateCall(SqlStdOperatorTable.MAX, false, null, name, child.toRexNode) } + + override def resultType = child.resultType + + override def validateInput = TypeCheckUtils.assertOrderableExpr(child.resultType, "max") } case class Count(child: Expression) extends Aggregation { - override def toString = s"($child).count" + override def toString = s"count($child)" override def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = { relBuilder.aggregateCall(SqlStdOperatorTable.COUNT, false, null, name, child.toRexNode) } + + override def resultType = BasicTypeInfo.LONG_TYPE_INFO } case class Avg(child: Expression) extends Aggregation { - override def toString = s"($child).avg" + override def toString = s"avg($child)" override def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = { relBuilder.aggregateCall(SqlStdOperatorTable.AVG, false, null, name, child.toRexNode) } + + override def resultType = child.resultType + + override def validateInput = TypeCheckUtils.assertNumericExpr(child.resultType, "avg") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala index ca67697197562..0ce4685eb5469 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala @@ -25,15 +25,34 @@ import org.apache.calcite.sql.SqlOperator import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.tools.RelBuilder -import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.table.typeutils.TypeConverter +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, NumericTypeInfo, TypeInformation} +import org.apache.flink.api.table.typeutils.{TypeCheckUtils, TypeCoercion, TypeConverter} +import org.apache.flink.api.table.validate._ -abstract class BinaryArithmetic extends BinaryExpression { self: Product => +abstract class BinaryArithmetic extends BinaryExpression { def sqlOperator: SqlOperator override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.call(sqlOperator, children.map(_.toRexNode)) } + + override def resultType: TypeInformation[_] = + TypeCoercion.widerTypeOf(left.resultType, right.resultType) match { + case Some(t) => t + case None => + throw new RuntimeException("This should never happen.") + } + + // TODO: tighten this rule once we implemented type coercion rules during validation + override def validateInput(): ExprValidationResult = { + if (!left.resultType.isInstanceOf[NumericTypeInfo[_]] || + !right.resultType.isInstanceOf[NumericTypeInfo[_]]) { + ValidationFailure(s"$this requires both operands Numeric, get" + + s"${left.resultType} and ${right.resultType}") + } else { + ValidationSuccess + } + } } case class Plus(left: Expression, right: Expression) extends BinaryArithmetic { @@ -56,6 +75,20 @@ case class Plus(left: Expression, right: Expression) extends BinaryArithmetic { relBuilder.call(SqlStdOperatorTable.PLUS, l, r) } } + + // TODO: tighten this rule once we implemented type coercion rules during validation + override def validateInput(): ExprValidationResult = { + if (left.resultType == BasicTypeInfo.STRING_TYPE_INFO || + right.resultType == BasicTypeInfo.STRING_TYPE_INFO) { + ValidationSuccess + } else if (!left.resultType.isInstanceOf[NumericTypeInfo[_]] || + !right.resultType.isInstanceOf[NumericTypeInfo[_]]) { + ValidationFailure(s"$this requires Numeric or String input," + + s" get ${left.resultType} and ${right.resultType}") + } else { + ValidationSuccess + } + } } case class UnaryMinus(child: Expression) extends UnaryExpression { @@ -64,6 +97,11 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.call(SqlStdOperatorTable.UNARY_MINUS, child.toRexNode) } + + override def resultType = child.resultType + + override def validateInput(): ExprValidationResult = + TypeCheckUtils.assertNumericExpr(child.resultType, "unary minus") } case class Minus(left: Expression, right: Expression) extends BinaryArithmetic { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala index e36a784e232c6..bf2e6bad21ccc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala @@ -18,85 +18,28 @@ package org.apache.flink.api.table.expressions import org.apache.calcite.rex.RexNode -import org.apache.calcite.sql.SqlOperator -import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.tools.RelBuilder +import org.apache.flink.api.table.UnresolvedException +import org.apache.flink.api.table.validate.{ExprValidationResult, ValidationFailure} + /** * General expression for unresolved function calls. The function can be a built-in * scalar function or a user-defined scalar function. */ -case class Call(functionName: String, args: Expression*) extends Expression { +case class Call(functionName: String, args: Seq[Expression]) extends Expression { override def children: Seq[Expression] = args override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { - relBuilder.call( - BuiltInFunctionNames.toSqlOperator(functionName), - args.map(_.toRexNode): _*) + throw new UnresolvedException(s"trying to convert UnresolvedFunction $functionName to RexNode") } override def toString = s"\\$functionName(${args.mkString(", ")})" - override def makeCopy(newArgs: Seq[AnyRef]): this.type = { - val copy = Call( - newArgs.head.asInstanceOf[String], - newArgs.drop(1).asInstanceOf[Seq[Expression]]: _*) - - copy.asInstanceOf[this.type] - } -} - -/** - * Enumeration of common function names. - */ -object BuiltInFunctionNames { - val SUBSTRING = "SUBSTRING" - val TRIM = "TRIM" - val CHAR_LENGTH = "CHARLENGTH" - val UPPER_CASE = "UPPERCASE" - val LOWER_CASE = "LOWERCASE" - val INIT_CAP = "INITCAP" - val LIKE = "LIKE" - val SIMILAR = "SIMILAR" - val MOD = "MOD" - val EXP = "EXP" - val LOG10 = "LOG10" - val POWER = "POWER" - val LN = "LN" - val ABS = "ABS" - val FLOOR = "FLOOR" - val CEIL = "CEIL" - - def toSqlOperator(name: String): SqlOperator = { - name match { - case BuiltInFunctionNames.SUBSTRING => SqlStdOperatorTable.SUBSTRING - case BuiltInFunctionNames.TRIM => SqlStdOperatorTable.TRIM - case BuiltInFunctionNames.CHAR_LENGTH => SqlStdOperatorTable.CHAR_LENGTH - case BuiltInFunctionNames.UPPER_CASE => SqlStdOperatorTable.UPPER - case BuiltInFunctionNames.LOWER_CASE => SqlStdOperatorTable.LOWER - case BuiltInFunctionNames.INIT_CAP => SqlStdOperatorTable.INITCAP - case BuiltInFunctionNames.LIKE => SqlStdOperatorTable.LIKE - case BuiltInFunctionNames.SIMILAR => SqlStdOperatorTable.SIMILAR_TO - case BuiltInFunctionNames.EXP => SqlStdOperatorTable.EXP - case BuiltInFunctionNames.LOG10 => SqlStdOperatorTable.LOG10 - case BuiltInFunctionNames.POWER => SqlStdOperatorTable.POWER - case BuiltInFunctionNames.LN => SqlStdOperatorTable.LN - case BuiltInFunctionNames.ABS => SqlStdOperatorTable.ABS - case BuiltInFunctionNames.MOD => SqlStdOperatorTable.MOD - case BuiltInFunctionNames.FLOOR => SqlStdOperatorTable.FLOOR - case BuiltInFunctionNames.CEIL => SqlStdOperatorTable.CEIL - case _ => ??? - } - } -} + override def resultType = + throw new UnresolvedException(s"calling resultType on UnresolvedFunction $functionName") -/** - * Enumeration of common function flags. - */ -object BuiltInFunctionConstants { - val TRIM_BOTH = Literal(0) - val TRIM_LEADING = Literal(1) - val TRIM_TRAILING = Literal(2) - val TRIM_DEFAULT_CHAR = Literal(" ") + override def validateInput(): ExprValidationResult = + ValidationFailure(s"Unresolved function call: $functionName") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala index fdad1f6b5c252..3b8b0e7bf4523 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala @@ -21,18 +21,27 @@ import org.apache.calcite.rex.RexNode import org.apache.calcite.tools.RelBuilder import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.table.typeutils.TypeConverter +import org.apache.flink.api.table.typeutils.{TypeCoercion, TypeConverter} +import org.apache.flink.api.table.validate._ -case class Cast(child: Expression, tpe: TypeInformation[_]) extends UnaryExpression { +case class Cast(child: Expression, resultType: TypeInformation[_]) extends UnaryExpression { - override def toString = s"$child.cast($tpe)" + override def toString = s"$child.cast($resultType)" override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { - relBuilder.cast(child.toRexNode, TypeConverter.typeInfoToSqlType(tpe)) + relBuilder.cast(child.toRexNode, TypeConverter.typeInfoToSqlType(resultType)) } - override def makeCopy(anyRefs: Seq[AnyRef]): this.type = { + override def makeCopy(anyRefs: Array[AnyRef]): this.type = { val child: Expression = anyRefs.head.asInstanceOf[Expression] - copy(child, tpe).asInstanceOf[this.type] + copy(child, resultType).asInstanceOf[this.type] + } + + override def validateInput(): ExprValidationResult = { + if (TypeCoercion.canCast(child.resultType, resultType)) { + ValidationSuccess + } else { + ValidationFailure(s"Unsupported cast from ${child.resultType} to $resultType") + } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala index 124393cced087..63caeaa43d301 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala @@ -24,24 +24,59 @@ import org.apache.calcite.sql.SqlOperator import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.tools.RelBuilder -abstract class BinaryComparison extends BinaryExpression { self: Product => +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.common.typeinfo.NumericTypeInfo +import org.apache.flink.api.table.validate._ + +abstract class BinaryComparison extends BinaryExpression { def sqlOperator: SqlOperator override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.call(sqlOperator, children.map(_.toRexNode)) } + + override def resultType = BOOLEAN_TYPE_INFO + + // TODO: tighten this rule once we implemented type coercion rules during validation + override def validateInput(): ExprValidationResult = (left.resultType, right.resultType) match { + case (STRING_TYPE_INFO, STRING_TYPE_INFO) => ValidationSuccess + case (_: NumericTypeInfo[_], _: NumericTypeInfo[_]) => ValidationSuccess + case (lType, rType) => + ValidationFailure( + s"Comparison is only supported for Strings and numeric types, get $lType and $rType") + } } case class EqualTo(left: Expression, right: Expression) extends BinaryComparison { override def toString = s"$left === $right" val sqlOperator: SqlOperator = SqlStdOperatorTable.EQUALS + + override def validateInput(): ExprValidationResult = (left.resultType, right.resultType) match { + case (_: NumericTypeInfo[_], _: NumericTypeInfo[_]) => ValidationSuccess + case (lType, rType) => + if (lType != rType) { + ValidationFailure(s"Equality predicate on incompatible types: $lType and $rType") + } else { + ValidationSuccess + } + } } case class NotEqualTo(left: Expression, right: Expression) extends BinaryComparison { override def toString = s"$left !== $right" val sqlOperator: SqlOperator = SqlStdOperatorTable.NOT_EQUALS + + override def validateInput(): ExprValidationResult = (left.resultType, right.resultType) match { + case (_: NumericTypeInfo[_], _: NumericTypeInfo[_]) => ValidationSuccess + case (lType, rType) => + if (lType != rType) { + ValidationFailure(s"Equality predicate on incompatible types: $lType and $rType") + } else { + ValidationSuccess + } + } } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { @@ -74,6 +109,8 @@ case class IsNull(child: Expression) extends UnaryExpression { override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.isNull(child.toRexNode) } + + override def resultType = BOOLEAN_TYPE_INFO } case class IsNotNull(child: Expression) extends UnaryExpression { @@ -82,4 +119,6 @@ case class IsNotNull(child: Expression) extends UnaryExpression { override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.isNotNull(child.toRexNode) } + + override def resultType = BOOLEAN_TYPE_INFO } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala index 82f76538844a4..24283d0dcdf37 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala @@ -20,27 +20,88 @@ package org.apache.flink.api.table.expressions import org.apache.calcite.rex.RexNode import org.apache.calcite.tools.RelBuilder -case class UnresolvedFieldReference(override val name: String) extends LeafExpression { +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.UnresolvedException +import org.apache.flink.api.table.validate.{ExprValidationResult, ValidationFailure} + +trait NamedExpression extends Expression { + def name: String + def toAttribute: Attribute +} + +abstract class Attribute extends LeafExpression with NamedExpression { + override def toAttribute: Attribute = this + + def withName(newName: String): Attribute +} + +case class UnresolvedFieldReference(name: String) extends Attribute { + override def toString = "\"" + name + override def withName(newName: String): Attribute = UnresolvedFieldReference(newName) + + override def resultType: TypeInformation[_] = + throw new UnresolvedException(s"calling resultType on ${this.getClass}") + + override def validateInput(): ExprValidationResult = + ValidationFailure(s"Unresolved reference $name") +} + +case class ResolvedFieldReference( + name: String, + resultType: TypeInformation[_]) extends Attribute { + + override def toString = s"'$name" + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.field(name) } -} -case class ResolvedFieldReference(override val name: String) extends LeafExpression { - override def toString = s"'$name" + override def withName(newName: String): Attribute = { + if (newName == name) { + this + } else { + ResolvedFieldReference(newName, resultType) + } + } } -case class Naming(child: Expression, override val name: String) extends UnaryExpression { +case class Alias(child: Expression, name: String) + extends UnaryExpression with NamedExpression { + override def toString = s"$child as '$name" override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.alias(child.toRexNode, name) } - override def makeCopy(anyRefs: Seq[AnyRef]): this.type = { + override def resultType: TypeInformation[_] = child.resultType + + override def makeCopy(anyRefs: Array[AnyRef]): this.type = { val child: Expression = anyRefs.head.asInstanceOf[Expression] copy(child, name).asInstanceOf[this.type] } + + override def toAttribute: Attribute = { + if (valid) { + ResolvedFieldReference(name, child.resultType) + } else { + UnresolvedFieldReference(name) + } + } +} + +case class UnresolvedAlias(child: Expression) extends UnaryExpression with NamedExpression { + + override def name: String = + throw new UnresolvedException("Invalid call to name on UnresolvedAlias") + + override def toAttribute: Attribute = + throw new UnresolvedException("Invalid call to toAttribute on UnresolvedAlias") + + override def resultType: TypeInformation[_] = + throw new UnresolvedException("Invalid call to resultType on UnresolvedAlias") + + override lazy val valid = false } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala index 1fbe5a3709073..9caec26bd556e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala @@ -22,7 +22,6 @@ import java.util.Date import org.apache.calcite.rex.RexNode import org.apache.calcite.tools.RelBuilder import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.api.scala.table.ImplicitExpressionOperations import org.apache.flink.api.table.typeutils.TypeConverter object Literal { @@ -39,11 +38,7 @@ object Literal { } } -case class Literal(value: Any, tpe: TypeInformation[_]) - extends LeafExpression with ImplicitExpressionOperations { - def expr = this - def typeInfo = tpe - +case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpression { override def toString = s"$value" override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { @@ -51,13 +46,10 @@ case class Literal(value: Any, tpe: TypeInformation[_]) } } -case class Null(tpe: TypeInformation[_]) extends LeafExpression { - def expr = this - def typeInfo = tpe - +case class Null(resultType: TypeInformation[_]) extends LeafExpression { override def toString = s"null" override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { - relBuilder.getRexBuilder.makeNullLiteral(TypeConverter.typeInfoToSqlType(tpe)) + relBuilder.getRexBuilder.makeNullLiteral(TypeConverter.typeInfoToSqlType(resultType)) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala index 37a659710d465..90d3dbcbe6127 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala @@ -21,25 +21,47 @@ import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.tools.RelBuilder -abstract class BinaryPredicate extends BinaryExpression { self: Product => } +import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.api.table.validate._ + +abstract class BinaryPredicate extends BinaryExpression { + override def resultType = BasicTypeInfo.BOOLEAN_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (left.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO && + right.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"$this only accepts children of Boolean Type, " + + s"get ${left.resultType} and ${right.resultType}") + } + } +} case class Not(child: Expression) extends UnaryExpression { - override val name = Expression.freshName("not-" + child.name) - override def toString = s"!($child)" override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.not(child.toRexNode) } + + override def resultType = BasicTypeInfo.BOOLEAN_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (child.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"Not only accepts child of Boolean Type, " + + s"get ${child.resultType}") + } + } } case class And(left: Expression, right: Expression) extends BinaryPredicate { override def toString = s"$left && $right" - override val name = Expression.freshName(left.name + "-and-" + right.name) - override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.and(left.toRexNode, right.toRexNode) } @@ -49,8 +71,6 @@ case class Or(left: Expression, right: Expression) extends BinaryPredicate { override def toString = s"$left || $right" - override val name = Expression.freshName(left.name + "-or-" + right.name) - override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.or(left.toRexNode, right.toRexNode) } @@ -63,10 +83,9 @@ case class Eval( extends Expression { def children = Seq(condition, ifTrue, ifFalse) - override def toString = s"($condition)? $ifTrue : $ifFalse" + override def resultType = ifTrue.resultType - override val name = Expression.freshName("if-" + condition.name + - "-then-" + ifTrue.name + "-else-" + ifFalse.name) + override def toString = s"($condition)? $ifTrue : $ifFalse" override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { val c = condition.toRexNode @@ -74,4 +93,15 @@ case class Eval( val f = ifFalse.toRexNode relBuilder.call(SqlStdOperatorTable.CASE, c, t, f) } + + override def validateInput(): ExprValidationResult = { + if (condition.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO && + ifTrue.resultType == ifFalse.resultType) { + ValidationSuccess + } else { + ValidationFailure( + s"Eval should have boolean condition and same type of ifTrue and ifFalse, get " + + s"(${condition.resultType}, ${ifTrue.resultType}, ${ifFalse.resultType})") + } + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala new file mode 100644 index 0000000000000..cf734f0e1dd8b --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.table.expressions + +import org.apache.calcite.rex.RexNode +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.tools.RelBuilder + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.typeutils.TypeCheckUtils +import org.apache.flink.api.table.validate._ + +case class Abs(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = child.resultType + + override def validateInput(): ExprValidationResult = + TypeCheckUtils.assertNumericExpr(child.resultType, "Abs") + + override def toString(): String = s"abs($child)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.ABS, child.toRexNode) + } +} + +case class Ceil(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = LONG_TYPE_INFO + + override def validateInput(): ExprValidationResult = + TypeCheckUtils.assertNumericExpr(child.resultType, "Ceil") + + override def toString(): String = s"ceil($child)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.CEIL, child.toRexNode) + } +} + +case class Exp(child: Expression) extends UnaryExpression with InputTypeSpec { + override def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO + + override def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil + + override def toString(): String = s"exp($child)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.EXP, child.toRexNode) + } +} + + +case class Floor(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = LONG_TYPE_INFO + + override def validateInput(): ExprValidationResult = + TypeCheckUtils.assertNumericExpr(child.resultType, "Floor") + + override def toString(): String = s"floor($child)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.FLOOR, child.toRexNode) + } +} + +case class Log10(child: Expression) extends UnaryExpression with InputTypeSpec { + override def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO + + override def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil + + override def toString(): String = s"log10($child)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.LOG10, child.toRexNode) + } +} + +case class Ln(child: Expression) extends UnaryExpression with InputTypeSpec { + override def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO + + override def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil + + override def toString(): String = s"ln($child)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.LN, child.toRexNode) + } +} + +case class Power(left: Expression, right: Expression) extends BinaryExpression with InputTypeSpec { + override def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO + + override def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: DOUBLE_TYPE_INFO :: Nil + + override def toString(): String = s"pow($left, $right)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.POWER, left.toRexNode, right.toRexNode) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala index 75fa07829ef07..887cf60b15613 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala @@ -16,28 +16,39 @@ * limitations under the License. */ package org.apache.flink.api.table.expressions + import org.apache.calcite.rex.RexNode import org.apache.calcite.tools.RelBuilder -abstract class Ordering extends UnaryExpression { self: Product => +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.validate._ + +abstract class Ordering extends UnaryExpression { + override def validateInput(): ExprValidationResult = { + if (!child.isInstanceOf[NamedExpression]) { + ValidationFailure(s"Sort should only based on field reference") + } else { + ValidationSuccess + } + } } case class Asc(child: Expression) extends Ordering { override def toString: String = s"($child).asc" - override def name: String = child.name + "-asc" - override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { child.toRexNode } + + override def resultType: TypeInformation[_] = child.resultType } case class Desc(child: Expression) extends Ordering { override def toString: String = s"($child).desc" - override def name: String = child.name + "-desc" - override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { relBuilder.desc(child.toRexNode) } + + override def resultType: TypeInformation[_] = child.resultType } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala new file mode 100644 index 0000000000000..11825f75f3acb --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.table.expressions + +import scala.collection.JavaConversions._ + +import org.apache.calcite.rex.RexNode +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.tools.RelBuilder + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.table.validate._ + +/** + * Returns the length of this `str`. + */ +case class CharLength(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = INT_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (child.resultType == STRING_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"CharLength only accepts String input, get ${child.resultType}") + } + } + + override def toString(): String = s"($child).charLength()" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.CHAR_LENGTH, child.toRexNode) + } +} + +/** + * Returns str with the first letter of each word in uppercase. + * All other letters are in lowercase. Words are delimited by white space. + */ +case class InitCap(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = STRING_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (child.resultType == STRING_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"InitCap only accepts String input, get ${child.resultType}") + } + } + + override def toString(): String = s"($child).initCap()" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.INITCAP, child.toRexNode) + } +} + +/** + * Returns true if `str` matches `pattern`. + */ +case class Like(str: Expression, pattern: Expression) extends BinaryExpression { + def left: Expression = str + def right: Expression = pattern + + override def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (str.resultType == STRING_TYPE_INFO && pattern.resultType == STRING_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"Like only accepts (String, String) input, " + + s"get (${str.resultType}, ${pattern.resultType})") + } + } + + override def toString(): String = s"($str).like($pattern)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.LIKE, children.map(_.toRexNode)) + } +} + +/** + * Returns str with all characters changed to lowercase. + */ +case class Lower(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = STRING_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (child.resultType == STRING_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"Lower only accepts String input, get ${child.resultType}") + } + } + + override def toString(): String = s"($child).toLowerCase()" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.LOWER, child.toRexNode) + } +} + +/** + * Returns true if `str` is similar to `pattern`. + */ +case class Similar(str: Expression, pattern: Expression) extends BinaryExpression { + def left: Expression = str + def right: Expression = pattern + + override def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (str.resultType == STRING_TYPE_INFO && pattern.resultType == STRING_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"Similar only accepts (String, String) input, " + + s"get (${str.resultType}, ${pattern.resultType})") + } + } + + override def toString(): String = s"($str).similarTo($pattern)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.SIMILAR_TO, children.map(_.toRexNode)) + } +} + +/** + * Returns subString of `str` from `begin`(inclusive) to `end`(not inclusive). + */ +case class SubString( + str: Expression, + begin: Expression, + end: Expression) extends Expression with InputTypeSpec { + + def this(str: Expression, begin: Expression) = this(str, begin, CharLength(str)) + + override def children: Seq[Expression] = str :: begin :: end :: Nil + + override def resultType: TypeInformation[_] = STRING_TYPE_INFO + + override def expectedTypes: Seq[TypeInformation[_]] = + Seq(STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO) + + override def toString(): String = s"$str.subString($begin, $end)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.SUBSTRING, children.map(_.toRexNode)) + } +} + +/** + * Trim `trimString` from `str` according to `trimFlag`: + * 0 for TRIM_BOTH, 1 for TRIM_LEADING and 2 for TRIM_TRAILING. + */ +case class Trim( + trimFlag: Expression, + trimString: Expression, + str: Expression) extends Expression with InputTypeSpec { + + override def children: Seq[Expression] = trimFlag :: trimString :: str :: Nil + + override def resultType: TypeInformation[_] = STRING_TYPE_INFO + + override def expectedTypes: Seq[TypeInformation[_]] = + Seq(INT_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO) + + override def toString(): String = s"trim($trimFlag, $trimString, $str)" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.TRIM, children.map(_.toRexNode)) + } +} + +/** + * Enumeration of trim flags. + */ +object TrimConstants { + val TRIM_BOTH = Literal(0) + val TRIM_LEADING = Literal(1) + val TRIM_TRAILING = Literal(2) + val TRIM_DEFAULT_CHAR = Literal(" ") +} + +/** + * Returns str with all characters changed to uppercase. + */ +case class Upper(child: Expression) extends UnaryExpression { + override def resultType: TypeInformation[_] = STRING_TYPE_INFO + + override def validateInput(): ExprValidationResult = { + if (child.resultType == STRING_TYPE_INFO) { + ValidationSuccess + } else { + ValidationFailure(s"Upper only accepts String input, get ${child.resultType}") + } + } + + override def toString(): String = s"($child).toUpperCase()" + + override def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.UPPER, child.toRexNode) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/RexNodeTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/RexNodeTranslator.scala index f946ed97b166e..095cf0486a30f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/RexNodeTranslator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/RexNodeTranslator.scala @@ -18,57 +18,49 @@ package org.apache.flink.api.table.plan -import org.apache.calcite.tools.RelBuilder.AggCall import org.apache.flink.api.table.TableEnvironment - import org.apache.flink.api.table.expressions._ object RexNodeTranslator { /** - * Extracts all aggregation expressions (zero, one, or more) from an expression, translates - * these aggregation expressions into Calcite AggCalls, and replaces the original aggregation - * expressions by field accesses expressions. + * Extracts all aggregation expressions (zero, one, or more) from an expression, + * and replaces the original aggregation expressions by field accesses expressions. */ - def extractAggCalls( + def extractAggregations( exp: Expression, - tableEnv: TableEnvironment): Pair[Expression, List[AggCall]] = { - - val relBuilder = tableEnv.getRelBuilder + tableEnv: TableEnvironment): Pair[Expression, List[NamedExpression]] = { exp match { case agg: Aggregation => val name = tableEnv.createUniqueAttributeName() - val aggCall = agg.toAggCall(name)(relBuilder) + val aggCall = Alias(agg, name) val fieldExp = new UnresolvedFieldReference(name) (fieldExp, List(aggCall)) - case n@Naming(agg: Aggregation, name) => - val aggCall = agg.toAggCall(name)(relBuilder) + case n @ Alias(agg: Aggregation, name) => val fieldExp = new UnresolvedFieldReference(name) - (fieldExp, List(aggCall)) + (fieldExp, List(n)) case l: LeafExpression => (l, Nil) case u: UnaryExpression => - val c = extractAggCalls(u.child, tableEnv) - (u.makeCopy(List(c._1)), c._2) + val c = extractAggregations(u.child, tableEnv) + (u.makeCopy(Array(c._1)), c._2) case b: BinaryExpression => - val l = extractAggCalls(b.left, tableEnv) - val r = extractAggCalls(b.right, tableEnv) - (b.makeCopy(List(l._1, r._1)), l._2 ::: r._2) - case e: Eval => - val c = extractAggCalls(e.condition, tableEnv) - val t = extractAggCalls(e.ifTrue, tableEnv) - val f = extractAggCalls(e.ifFalse, tableEnv) - (e.makeCopy(List(c._1, t._1, f._1)), c._2 ::: t._2 ::: f._2) + val l = extractAggregations(b.left, tableEnv) + val r = extractAggregations(b.right, tableEnv) + (b.makeCopy(Array(l._1, r._1)), l._2 ::: r._2) // Scalar functions - case c@Call(name, args@_*) => - val newArgs = args.map(extractAggCalls(_, tableEnv)).toList - (c.makeCopy(name :: newArgs.map(_._1)), newArgs.flatMap(_._2)) + case c @ Call(name, args) => + val newArgs = args.map(extractAggregations(_, tableEnv)) + (c.makeCopy((name :: newArgs.map(_._1) :: Nil).toArray), newArgs.flatMap(_._2).toList) - case e@AnyRef => - throw new IllegalArgumentException( - s"Expression $e of type ${e.getClass} not supported yet") + case e: Expression => + val newArgs = e.productIterator.map { + case arg: Expression => + extractAggregations(arg, tableEnv) + } + (e.makeCopy(newArgs.map(_._1).toArray), newArgs.flatMap(_._2).toList) } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala new file mode 100644 index 0000000000000..dae02bd3c64c6 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.table.plan.logical + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.tools.RelBuilder + +import org.apache.flink.api.table.{TableEnvironment, ValidationException} +import org.apache.flink.api.table.expressions._ +import org.apache.flink.api.table.trees.TreeNode +import org.apache.flink.api.table.typeutils.TypeCoercion +import org.apache.flink.api.table.validate._ + +/** + * LogicalNode is created and validated as we construct query plan using Table API. + * + * The main validation procedure is separated into two phases: + * + * Expressions' resolution and transformation ([[resolveExpressions]]): + * + * - translate [[UnresolvedFieldReference]] into [[ResolvedFieldReference]] + * using child operator's output + * - translate [[Call]](UnresolvedFunction) into solid Expression + * - generate alias names for query output + * - .... + * + * LogicalNode validation ([[validate]]): + * + * - check no [[UnresolvedFieldReference]] exists any more + * - check if all expressions have children of needed type + * - check each logical operator have desired input + * + * Once we pass the validation phase, we can safely convert LogicalNode into Calcite's RelNode. + */ +abstract class LogicalNode extends TreeNode[LogicalNode] { + def output: Seq[Attribute] + + def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = { + // resolve references and function calls + val exprResolved = expressionPostOrderTransform { + case u @ UnresolvedFieldReference(name) => + resolveReference(name).getOrElse(u) + case c @ Call(name, children) if c.childrenValid => + tableEnv.getFunctionCatalog.lookupFunction(name, children) + } + + exprResolved.expressionPostOrderTransform { + case ips: InputTypeSpec if ips.childrenValid => + var changed: Boolean = false + val newChildren = ips.expectedTypes.zip(ips.children).map { case (tpe, child) => + val childType = child.resultType + if (childType != tpe && TypeCoercion.canSafelyCast(childType, tpe)) { + changed = true + Cast(child, tpe) + } else { + child + } + }.toArray[AnyRef] + if (changed) ips.makeCopy(newChildren) else ips + } + } + + final def toRelNode(relBuilder: RelBuilder): RelNode = construct(relBuilder).build() + + protected[logical] def construct(relBuilder: RelBuilder): RelBuilder + + def validate(tableEnv: TableEnvironment): LogicalNode = { + val resolvedNode = resolveExpressions(tableEnv) + resolvedNode.expressionPostOrderTransform { + case a: Attribute if !a.valid => + val from = children.flatMap(_.output).map(_.name).mkString(", ") + failValidation(s"cannot resolve [${a.name}] given input [$from]") + + case e: Expression if e.validateInput().isFailure => + failValidation(s"Expression $e failed on input check: " + + s"${e.validateInput().asInstanceOf[ValidationFailure].message}") + } + } + + /** + * Resolves the given strings to a [[NamedExpression]] using the input from all child + * nodes of this LogicalPlan. + */ + def resolveReference(name: String): Option[NamedExpression] = { + val childrenOutput = children.flatMap(_.output) + val candidates = childrenOutput.filter(_.name.equalsIgnoreCase(name)) + if (candidates.length > 1) { + failValidation(s"Reference $name is ambiguous") + } else if (candidates.length == 0) { + None + } else { + Some(candidates.head.withName(name)) + } + } + + /** + * Runs [[postOrderTransform]] with `rule` on all expressions present in this logical node. + * + * @param rule the rule to be applied to every expression in this logical node. + */ + def expressionPostOrderTransform(rule: PartialFunction[Expression, Expression]): LogicalNode = { + var changed = false + + def expressionPostOrderTransform(e: Expression): Expression = { + val newExpr = e.postOrderTransform(rule) + if (newExpr.fastEquals(e)) { + e + } else { + changed = true + newExpr + } + } + + val newArgs = productIterator.map { + case e: Expression => expressionPostOrderTransform(e) + case Some(e: Expression) => Some(expressionPostOrderTransform(e)) + case seq: Traversable[_] => seq.map { + case e: Expression => expressionPostOrderTransform(e) + case other => other + } + case other: AnyRef => other + }.toArray + + if (changed) makeCopy(newArgs) else this + } + + protected def failValidation(msg: String): Nothing = { + throw new ValidationException(msg) + } +} + +abstract class LeafNode extends LogicalNode { + override def children: Seq[LogicalNode] = Nil +} + +abstract class UnaryNode extends LogicalNode { + def child: LogicalNode + + override def children: Seq[LogicalNode] = child :: Nil +} + +abstract class BinaryNode extends LogicalNode { + def left: LogicalNode + def right: LogicalNode + + override def children: Seq[LogicalNode] = left :: right :: Nil +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala new file mode 100644 index 0000000000000..d3476510e9253 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.table.plan.logical + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.JoinRelType +import org.apache.calcite.rel.logical.LogicalProject +import org.apache.calcite.tools.RelBuilder + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.java.operators.join.JoinType +import org.apache.flink.api.table._ +import org.apache.flink.api.table.expressions._ +import org.apache.flink.api.table.typeutils.TypeConverter + +case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extends UnaryNode { + override def output: Seq[Attribute] = projectList.map(_.toAttribute) + + override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = { + val afterResolve = super.resolveExpressions(tableEnv).asInstanceOf[Project] + val newProjectList = + afterResolve.projectList.zipWithIndex.map { case (e, i) => + e match { + case u @ UnresolvedAlias(child) => child match { + case ne: NamedExpression => ne + case e if !e.valid => u + case c @ Cast(ne: NamedExpression, tp) => Alias(c, s"${ne.name}-$tp") + case other => Alias(other, s"_c$i") + } + case _ => throw new IllegalArgumentException + } + } + Project(newProjectList, child) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + val resolvedProject = super.validate(tableEnv).asInstanceOf[Project] + + def checkUniqueNames(exprs: Seq[Expression]): Unit = { + val names: mutable.Set[String] = mutable.Set() + exprs.foreach { + case n: Alias => + // explicit name + if (names.contains(n.name)) { + throw new ValidationException(s"Duplicate field name $n.name.") + } else { + names.add(n.name) + } + case r: ResolvedFieldReference => + // simple field forwarding + if (names.contains(r.name)) { + throw new ValidationException(s"Duplicate field name $r.name.") + } else { + names.add(r.name) + } + case _ => // Do nothing + } + } + checkUniqueNames(resolvedProject.projectList) + resolvedProject + } + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + val allAlias = projectList.forall(_.isInstanceOf[Alias]) + child.construct(relBuilder) + if (allAlias) { + // Calcite's RelBuilder does not translate identity projects even if they rename fields. + // Add a projection ourselves (will be automatically removed by translation rules). + relBuilder.push( + LogicalProject.create(relBuilder.peek(), + projectList.map(_.toRexNode(relBuilder)).asJava, + projectList.map(_.name).asJava)) + } else { + relBuilder.project(projectList.map(_.toRexNode(relBuilder)): _*) + } + } +} + +case class AliasNode(aliasList: Seq[Expression], child: LogicalNode) extends UnaryNode { + override def output: Seq[Attribute] = + throw new UnresolvedException("Invalid call to output on AliasNode") + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = + throw new UnresolvedException("Invalid call to toRelNode on AliasNode") + + override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = { + if (aliasList.length > child.output.length) { + failValidation("Aliasing more fields than we actually have") + } else if (!aliasList.forall(_.isInstanceOf[UnresolvedFieldReference])) { + failValidation("Alias only accept name expressions as arguments") + } else { + val names = aliasList.map(_.asInstanceOf[UnresolvedFieldReference].name) + val input = child.output + Project( + names.zip(input).map { case (name, attr) => + Alias(attr, name)} ++ input.drop(names.length), child) + } + } +} + +case class Distinct(child: LogicalNode) extends UnaryNode { + override def output: Seq[Attribute] = child.output + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + child.construct(relBuilder) + relBuilder.distinct() + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + if (tableEnv.isInstanceOf[StreamTableEnvironment]) { + throw new TableException(s"Distinct on stream tables is currently not supported.") + } + this + } +} + +case class Sort(order: Seq[Ordering], child: LogicalNode) extends UnaryNode { + override def output: Seq[Attribute] = child.output + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + child.construct(relBuilder) + relBuilder.sort(order.map(_.toRexNode(relBuilder)).asJava) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + if (tableEnv.isInstanceOf[StreamTableEnvironment]) { + throw new TableException(s"Distinct on stream tables is currently not supported.") + } + super.validate(tableEnv) + } +} + +case class Filter(condition: Expression, child: LogicalNode) extends UnaryNode { + override def output: Seq[Attribute] = child.output + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + child.construct(relBuilder) + relBuilder.filter(condition.toRexNode(relBuilder)) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + val resolvedFilter = super.validate(tableEnv).asInstanceOf[Filter] + if (resolvedFilter.condition.resultType != BOOLEAN_TYPE_INFO) { + failValidation(s"filter expression ${resolvedFilter.condition} of" + + s" ${resolvedFilter.condition.resultType} is not a boolean") + } + resolvedFilter + } +} + +case class Aggregate( + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[NamedExpression], + child: LogicalNode) extends UnaryNode { + + override def output: Seq[Attribute] = { + (groupingExpressions ++ aggregateExpressions) map { agg => + agg match { + case ne: NamedExpression => ne.toAttribute + case e => Alias(e, e.toString).toAttribute + } + } + } + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + child.construct(relBuilder) + relBuilder.aggregate( + relBuilder.groupKey(groupingExpressions.map(_.toRexNode(relBuilder)).asJava), + aggregateExpressions.map { e => + e match { + case Alias(agg: Aggregation, name) => agg.toAggCall(name)(relBuilder) + case _ => throw new RuntimeException("This should never happen.") + } + }.asJava) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + if (tableEnv.isInstanceOf[StreamTableEnvironment]) { + throw new TableException(s"Aggregate on stream tables is currently not supported.") + } + + val resolvedAggregate = super.validate(tableEnv).asInstanceOf[Aggregate] + val groupingExprs = resolvedAggregate.groupingExpressions + val aggregateExprs = resolvedAggregate.aggregateExpressions + aggregateExprs.foreach(validateAggregateExpression) + groupingExprs.foreach(validateGroupingExpression) + + def validateAggregateExpression(expr: Expression): Unit = expr match { + // check no nested aggregation exists. + case aggExpr: Aggregation => + aggExpr.children.foreach { child => + child.preOrderVisit { + case agg: Aggregation => + failValidation( + "It's not allowed to use an aggregate function as " + + "input of another aggregate function") + case _ => // OK + } + } + case a: Attribute if !groupingExprs.exists(_.checkEquals(a)) => + failValidation( + s"expression '$a' is invalid because it is neither" + + " present in group by nor an aggregate function") + case e if groupingExprs.exists(_.checkEquals(e)) => // OK + case e => e.children.foreach(validateAggregateExpression) + } + + def validateGroupingExpression(expr: Expression): Unit = { + if (!expr.resultType.isKeyType) { + failValidation( + s"expression $expr cannot be used as a grouping expression " + + "because it's not a valid key type") + } + } + resolvedAggregate + } +} + +case class Union(left: LogicalNode, right: LogicalNode) extends BinaryNode { + override def output: Seq[Attribute] = left.output + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + left.construct(relBuilder) + right.construct(relBuilder) + relBuilder.union(true) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + val resolvedUnion = super.validate(tableEnv).asInstanceOf[Union] + if (left.output.length != right.output.length) { + failValidation(s"Union two table of different column sizes:" + + s" ${left.output.size} and ${right.output.size}") + } + val sameSchema = left.output.zip(right.output).forall { case (l, r) => + l.resultType == r.resultType && l.name == r.name } + if (!sameSchema) { + failValidation(s"Union two table of different schema:" + + s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" + + s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]") + } + resolvedUnion + } +} + +case class Join( + left: LogicalNode, + right: LogicalNode, + joinType: JoinType, + condition: Option[Expression]) extends BinaryNode { + + override def output: Seq[Attribute] = { + joinType match { + case JoinType.INNER => left.output ++ right.output + case j => throw new ValidationException(s"Unsupported JoinType: $j") + } + } + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + joinType match { + case JoinType.INNER => + left.construct(relBuilder) + right.construct(relBuilder) + relBuilder.join(JoinRelType.INNER, + condition.map(_.toRexNode(relBuilder)).getOrElse(relBuilder.literal(true))) + case _ => + throw new ValidationException(s"Unsupported JoinType: $joinType") + } + } + + private def ambiguousName: Set[String] = + left.output.map(_.name).toSet.intersect(right.output.map(_.name).toSet) + + override def validate(tableEnv: TableEnvironment): LogicalNode = { + if (tableEnv.isInstanceOf[StreamTableEnvironment]) { + throw new TableException(s"Join on stream tables is currently not supported.") + } + + val resolvedJoin = super.validate(tableEnv).asInstanceOf[Join] + if (!resolvedJoin.condition.forall(_.resultType == BOOLEAN_TYPE_INFO)) { + failValidation(s"filter expression ${resolvedJoin.condition} is not a boolean") + } else if (!ambiguousName.isEmpty) { + failValidation(s"join relations with ambiguous names: ${ambiguousName.mkString(", ")}") + } + resolvedJoin + } +} + +case class CatalogNode( + tableName: String, + rowType: RelDataType) extends LeafNode { + + val output: Seq[Attribute] = rowType.getFieldList.asScala.map { field => + ResolvedFieldReference( + field.getName, TypeConverter.sqlTypeToTypeInfo(field.getType.getSqlTypeName)) + } + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + relBuilder.scan(tableName) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = this +} + +/** + * Wrapper for valid logical plans generated from SQL String. + */ +case class LogicalRelNode( + relNode: RelNode) extends LeafNode { + + val output: Seq[Attribute] = relNode.getRowType.getFieldList.asScala.map { field => + ResolvedFieldReference( + field.getName, TypeConverter.sqlTypeToTypeInfo(field.getType.getSqlTypeName)) + } + + override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { + relBuilder.push(relNode) + } + + override def validate(tableEnv: TableEnvironment): LogicalNode = this +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala index 5356a9d0c6c48..4f111c9ac3529 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala @@ -17,28 +17,18 @@ */ package org.apache.flink.api.table +import scala.collection.JavaConverters._ + import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.`type`.RelDataTypeField -import org.apache.calcite.rel.core.JoinRelType -import org.apache.calcite.rel.logical.LogicalProject -import org.apache.calcite.rex.{RexCall, RexInputRef, RexLiteral, RexNode} -import org.apache.calcite.sql.SqlKind -import org.apache.calcite.tools.RelBuilder.{AggCall, GroupKey} -import org.apache.calcite.util.NlsString + import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.plan.RexNodeTranslator.extractAggCalls +import org.apache.flink.api.java.operators.join.JoinType +import org.apache.flink.api.table.plan.RexNodeTranslator.extractAggregations import org.apache.flink.api.table.expressions._ +import org.apache.flink.api.table.plan.logical._ import org.apache.flink.api.table.sinks.TableSink import org.apache.flink.api.table.typeutils.TypeConverter -import scala.collection.mutable -import scala.collection.JavaConverters._ - -case class BaseTable( - private[flink] val relNode: RelNode, - private[flink] val tableEnv: TableEnvironment) - /** * A Table is the core component of the Table API. * Similar to how the batch and streaming APIs have DataSet and DataStream, @@ -66,18 +56,16 @@ case class BaseTable( * in a Scala DSL or as an expression String. Please refer to the documentation for the expression * syntax. * - * @param relNode The root node of the relational Calcite [[RelNode]] tree. * @param tableEnv The [[TableEnvironment]] to which the table is bound. + * @param logicalPlan */ class Table( - private[flink] override val relNode: RelNode, - private[flink] override val tableEnv: TableEnvironment) - extends BaseTable(relNode, tableEnv) -{ + private[flink] val tableEnv: TableEnvironment, + private[flink] val logicalPlan: LogicalNode) { def relBuilder = tableEnv.getRelBuilder - def getRelNode: RelNode = relNode + def getRelNode: RelNode = logicalPlan.toRelNode(relBuilder) /** * Performs a selection operation. Similar to an SQL SELECT statement. The field expressions @@ -90,44 +78,17 @@ class Table( * }}} */ def select(fields: Expression*): Table = { - - checkUniqueNames(fields) - - relBuilder.push(relNode) - - // separate aggregations and selection expressions - val extractedAggCalls: List[(Expression, List[AggCall])] = fields - .map(extractAggCalls(_, tableEnv)).toList - - // get aggregation calls - val aggCalls: List[AggCall] = extractedAggCalls.flatMap(_._2) - - // apply aggregations - if (aggCalls.nonEmpty) { - // aggregation on stream table is not currently supported - tableEnv match { - case _: StreamTableEnvironment => - throw new TableException("Aggregation on stream tables is currently not supported.") - case _ => - val emptyKey: GroupKey = relBuilder.groupKey() - relBuilder.aggregate(emptyKey, aggCalls.toIterable.asJava) - } - } - - // get selection expressions - val exprs: List[RexNode] = extractedAggCalls.map(_._1.toRexNode(relBuilder)) - - relBuilder.project(exprs.toIterable.asJava) - val projected = relBuilder.build() - - if(relNode == projected) { - // Calcite's RelBuilder does not translate identity projects even if they rename fields. - // Add a projection ourselves (will be automatically removed by translation rules). - new Table(createRenamingProject(exprs), tableEnv) + val projectionOnAggregates = fields.map(extractAggregations(_, tableEnv)) + val aggregations = projectionOnAggregates.flatMap(_._2) + if (aggregations.nonEmpty) { + new Table(tableEnv, + Project(projectionOnAggregates.map(e => UnresolvedAlias(e._1)), + Aggregate(Nil, aggregations, logicalPlan).validate(tableEnv)).validate(tableEnv)) } else { - new Table(projected, tableEnv) + new Table(tableEnv, + Project( + projectionOnAggregates.map(e => UnresolvedAlias(e._1)), logicalPlan).validate(tableEnv)) } - } /** @@ -156,30 +117,7 @@ class Table( * }}} */ def as(fields: Expression*): Table = { - - val curNames = relNode.getRowType.getFieldNames.asScala - - // validate that AS has only field references - if (! fields.forall( _.isInstanceOf[UnresolvedFieldReference] )) { - throw new IllegalArgumentException("All expressions must be field references.") - } - // validate that we have not more field references than fields - if ( fields.length > curNames.size) { - throw new IllegalArgumentException("More field references than fields.") - } - - val curFields = curNames.map(new UnresolvedFieldReference(_)) - - val renamings = fields.zip(curFields).map { - case (newName, oldName) => new Naming(oldName, newName.name) - } - val remaining = curFields.drop(fields.size) - - relBuilder.push(relNode) - - val exprs = (renamings ++ remaining).map(_.toRexNode(relBuilder)) - - new Table(createRenamingProject(exprs), tableEnv) + new Table(tableEnv, AliasNode(fields, logicalPlan).validate(tableEnv)) } /** @@ -208,11 +146,7 @@ class Table( * }}} */ def filter(predicate: Expression): Table = { - - relBuilder.push(relNode) - relBuilder.filter(predicate.toRexNode(relBuilder)) - - new Table(relBuilder.build(), tableEnv) + new Table(tableEnv, Filter(predicate, logicalPlan).validate(tableEnv)) } /** @@ -269,19 +203,10 @@ class Table( * }}} */ def groupBy(fields: Expression*): GroupedTable = { - - // group by on stream tables is currently not supported - tableEnv match { - case _: StreamTableEnvironment => - throw new TableException("Group by on stream tables is currently not supported.") - case _ => { - relBuilder.push(relNode) - val groupExpr = fields.map(_.toRexNode(relBuilder)).toIterable.asJava - val groupKey = relBuilder.groupKey(groupExpr) - - new GroupedTable(relBuilder.build(), tableEnv, groupKey) - } + if (tableEnv.isInstanceOf[StreamTableEnvironment]) { + throw new TableException(s"Group by on stream tables is currently not supported.") } + new GroupedTable(this, fields) } /** @@ -309,15 +234,7 @@ class Table( * }}} */ def distinct(): Table = { - // distinct on stream table is not currently supported - tableEnv match { - case _: StreamTableEnvironment => - throw new TableException("Distinct on stream tables is currently not supported.") - case _ => - relBuilder.push(relNode) - relBuilder.distinct() - new Table(relBuilder.build(), tableEnv) - } + new Table(tableEnv, Distinct(logicalPlan).validate(tableEnv)) } /** @@ -334,32 +251,12 @@ class Table( * }}} */ def join(right: Table): Table = { - - // join on stream tables is currently not supported - tableEnv match { - case _: StreamTableEnvironment => - throw new TableException("Join on stream tables is currently not supported.") - case _ => { - // check that right table belongs to the same TableEnvironment - if (right.tableEnv != this.tableEnv) { - throw new TableException("Only tables from the same TableEnvironment can be joined.") - } - - // check that join inputs do not have overlapping field names - val leftFields = relNode.getRowType.getFieldNames.asScala.toSet - val rightFields = right.relNode.getRowType.getFieldNames.asScala.toSet - if (leftFields.intersect(rightFields).nonEmpty) { - throw new IllegalArgumentException("Overlapping fields names on join input.") - } - - relBuilder.push(relNode) - relBuilder.push(right.relNode) - - relBuilder.join(JoinRelType.INNER, relBuilder.literal(true)) - val join = relBuilder.build() - new Table(join, tableEnv) - } + // check that right table belongs to the same TableEnvironment + if (right.tableEnv != this.tableEnv) { + throw new ValidationException("Only tables from the same TableEnvironment can be joined.") } + new Table(tableEnv, + Join(this.logicalPlan, right.logicalPlan, JoinType.INNER, None).validate(tableEnv)) } /** @@ -375,32 +272,11 @@ class Table( * }}} */ def unionAll(right: Table): Table = { - // check that right table belongs to the same TableEnvironment if (right.tableEnv != this.tableEnv) { - throw new TableException("Only tables from the same TableEnvironment can be unioned.") - } - - val leftRowType: List[RelDataTypeField] = relNode.getRowType.getFieldList.asScala.toList - val rightRowType: List[RelDataTypeField] = right.relNode.getRowType.getFieldList.asScala.toList - - if (leftRowType.length != rightRowType.length) { - throw new IllegalArgumentException("Unioned tables have varying row schema.") + throw new ValidationException("Only tables from the same TableEnvironment can be unioned.") } - else { - val zipped: List[(RelDataTypeField, RelDataTypeField)] = leftRowType.zip(rightRowType) - zipped.foreach { case (x, y) => - if (!x.getName.equals(y.getName) || x.getType != y.getType) { - throw new IllegalArgumentException("Unioned tables have varying row schema.") - } - } - } - - relBuilder.push(relNode) - relBuilder.push(right.relNode) - - relBuilder.union(true) - new Table(relBuilder.build(), tableEnv) + new Table(tableEnv, Union(logicalPlan, right.logicalPlan).validate(tableEnv)) } /** @@ -414,22 +290,13 @@ class Table( * }}} */ def orderBy(fields: Expression*): Table = { - relBuilder.push(relNode) - - if (! fields.forall { - case x : UnresolvedFieldReference => true - case x : Ordering => x.child.isInstanceOf[UnresolvedFieldReference] - case _ => false - }) { - throw new IllegalArgumentException("All expressions must be field references " + - "or asc/desc expressions.") + val order: Seq[Ordering] = fields.map { case e => + e match { + case o: Ordering => o + case _ => Asc(e) + } } - - val exprs = fields.map(_.toRexNode(relBuilder)) - - relBuilder.sort(exprs.asJava) - new Table(relBuilder.build(), tableEnv) - + new Table(tableEnv, Sort(order, logicalPlan).validate(tableEnv)) } /** @@ -460,7 +327,7 @@ class Table( def toSink[T](sink: TableSink[T]): Unit = { // get schema information of table - val rowType = relNode.getRowType + val rowType = getRelNode.getRowType val fieldNames: Array[String] = rowType.getFieldNames.asScala.toArray val fieldTypes: Array[TypeInformation[_]] = rowType.getFieldList.asScala .map(f => TypeConverter.sqlTypeToTypeInfo(f.getType.getSqlTypeName)).toArray @@ -471,62 +338,14 @@ class Table( // emit the table to the configured table sink tableEnv.emitToSink(this, configuredSink) } - - private def createRenamingProject(exprs: Seq[RexNode]): LogicalProject = { - - val names = exprs.map{ e => - e.getKind match { - case SqlKind.AS => - e.asInstanceOf[RexCall].getOperands.get(1) - .asInstanceOf[RexLiteral].getValue - .asInstanceOf[NlsString].getValue - case SqlKind.INPUT_REF => - relNode.getRowType.getFieldNames.get(e.asInstanceOf[RexInputRef].getIndex) - case _ => - throw new PlanGenException("Unexpected expression type encountered.") - } - - } - LogicalProject.create(relNode, exprs.toList.asJava, names.toList.asJava) - } - - private def checkUniqueNames(exprs: Seq[Expression]): Unit = { - val names: mutable.Set[String] = mutable.Set() - - exprs.foreach { - case n: Naming => - // explicit name - if (names.contains(n.name)) { - throw new IllegalArgumentException(s"Duplicate field name $n.name.") - } else { - names.add(n.name) - } - case u: UnresolvedFieldReference => - // simple field forwarding - if (names.contains(u.name)) { - throw new IllegalArgumentException(s"Duplicate field name $u.name.") - } else { - names.add(u.name) - } - case _ => // Do nothing - } - } - } /** * A table that has been grouped on a set of grouping keys. - * - * @param relNode The root node of the relational Calcite [[RelNode]] tree. - * @param tableEnv The [[TableEnvironment]] to which the table is bound. - * @param groupKey The Calcite [[GroupKey]] of this table. */ class GroupedTable( - private[flink] override val relNode: RelNode, - private[flink] override val tableEnv: TableEnvironment, - private[flink] val groupKey: GroupKey) extends BaseTable(relNode, tableEnv) { - - def relBuilder = tableEnv.getRelBuilder + private[flink] val table: Table, + private[flink] val groupKey: Seq[Expression]) { /** * Performs a selection operation on a grouped table. Similar to an SQL SELECT statement. @@ -540,31 +359,19 @@ class GroupedTable( */ def select(fields: Expression*): Table = { - relBuilder.push(relNode) - - // separate aggregations and selection expressions - val extractedAggCalls: List[(Expression, List[AggCall])] = fields - .map(extractAggCalls(_, tableEnv)).toList - - // get aggregation calls - val aggCalls: List[AggCall] = extractedAggCalls.flatMap(_._2) + val projectionOnAggregates = fields.map(extractAggregations(_, table.tableEnv)) + val aggregations = projectionOnAggregates.flatMap(_._2) - // apply aggregations - relBuilder.aggregate(groupKey, aggCalls.toIterable.asJava) - - // get selection expressions - val exprs: List[RexNode] = try { - extractedAggCalls.map(_._1.toRexNode(relBuilder)) - } catch { - case iae: IllegalArgumentException => - throw new IllegalArgumentException( - "Only grouping fields and aggregations allowed after groupBy.", iae) - case e: Exception => throw e + val logical = if (aggregations.nonEmpty) { + Project(projectionOnAggregates.map(e => UnresolvedAlias(e._1)), + Aggregate(groupKey, aggregations, table.logicalPlan).validate(table.tableEnv) + ) + } else { + Project(projectionOnAggregates.map(e => UnresolvedAlias(e._1)), + Aggregate(groupKey, Nil, table.logicalPlan).validate(table.tableEnv)) } - relBuilder.project(exprs.toIterable.asJava) - - new Table(relBuilder.build(), tableEnv) + new Table(table.tableEnv, logical.validate(table.tableEnv)) } /** @@ -581,5 +388,4 @@ class GroupedTable( val fieldExprs = ExpressionParser.parseExpressionList(fields) select(fieldExprs: _*) } - } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala new file mode 100644 index 0000000000000..63c7013d11b39 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.table.trees + +import org.apache.commons.lang.ClassUtils + +/** + * Generic base class for trees that can be transformed and traversed. + */ +abstract class TreeNode[A <: TreeNode[A]] extends Product { self: A => + + /** + * List of child nodes that should be considered when doing transformations. Other values + * in the Product will not be transformed, only handed through. + */ + def children: Seq[A] + + /** + * Tests for equality by first testing for reference equality. + */ + def fastEquals(other: TreeNode[_]): Boolean = this.eq(other) || this == other + + /** + * Do tree transformation in post order. + */ + def postOrderTransform(rule: PartialFunction[A, A]): A = { + def childrenTransform(rule: PartialFunction[A, A]): A = { + var changed = false + val newArgs = productIterator.map { + case arg: TreeNode[_] if children.contains(arg) => + val newChild = arg.asInstanceOf[A].postOrderTransform(rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case args: Traversable[_] => args.map { + case arg: TreeNode[_] if children.contains(arg) => + val newChild = arg.asInstanceOf[A].postOrderTransform(rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case other => other + } + case nonChild: AnyRef => nonChild + case null => null + }.toArray + if (changed) makeCopy(newArgs) else this + } + + val afterChildren = childrenTransform(rule) + if (afterChildren fastEquals this) { + rule.applyOrElse(this, identity[A]) + } else { + rule.applyOrElse(afterChildren, identity[A]) + } + } + + /** + * Runs the given function first on the node and then recursively on all its children. + */ + def preOrderVisit(f: A => Unit): Unit = { + f(this) + children.foreach(_.preOrderVisit(f)) + } + + /** + * Creates a new copy of this expression with new children. This is used during transformation + * if children change. + */ + def makeCopy(newArgs: Array[AnyRef]): A = { + val ctors = getClass.getConstructors.filter(_.getParameterTypes.size > 0) + if (ctors.isEmpty) { + throw new RuntimeException(s"No valid constructor for ${getClass.getSimpleName}") + } + + val defaultCtor = ctors.find { ctor => + if (ctor.getParameterTypes.size != newArgs.length) { + false + } else if (newArgs.contains(null)) { + false + } else { + val argsClasses: Array[Class[_]] = newArgs.map(_.getClass) + ClassUtils.isAssignable(argsClasses, ctor.getParameterTypes) + } + }.getOrElse(ctors.maxBy(_.getParameterTypes.size)) + + try { + defaultCtor.newInstance(newArgs: _*).asInstanceOf[A] + } catch { + case e: java.lang.IllegalArgumentException => + throw new IllegalArgumentException(s"Fail to copy treeNode ${getClass.getName}") + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala new file mode 100644 index 0000000000000..1da1d2cf0a01d --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.table.typeutils + +import org.apache.flink.api.common.typeinfo.{NumericTypeInfo, TypeInformation} +import org.apache.flink.api.table.validate._ + +object TypeCheckUtils { + + def assertNumericExpr(dataType: TypeInformation[_], caller: String): ExprValidationResult = { + if (dataType.isInstanceOf[NumericTypeInfo[_]]) { + ValidationSuccess + } else { + ValidationFailure(s"$caller requires numeric types, get $dataType here") + } + } + + def assertOrderableExpr(dataType: TypeInformation[_], caller: String): ExprValidationResult = { + if (dataType.isSortKeyType) { + ValidationSuccess + } else { + ValidationFailure(s"$caller requires orderable types, get $dataType here") + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala new file mode 100644 index 0000000000000..218996da9a37c --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.typeutils + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.common.typeinfo.{NumericTypeInfo, TypeInformation} + +/** + * Utilities for type conversions. + */ +object TypeCoercion { + + val numericWideningPrecedence: IndexedSeq[TypeInformation[_]] = + IndexedSeq( + BYTE_TYPE_INFO, + SHORT_TYPE_INFO, + INT_TYPE_INFO, + LONG_TYPE_INFO, + FLOAT_TYPE_INFO, + DOUBLE_TYPE_INFO) + + def widerTypeOf(tp1: TypeInformation[_], tp2: TypeInformation[_]): Option[TypeInformation[_]] = { + (tp1, tp2) match { + case (tp1, tp2) if tp1 == tp2 => Some(tp1) + + case (_, STRING_TYPE_INFO) => Some(STRING_TYPE_INFO) + case (STRING_TYPE_INFO, _) => Some(STRING_TYPE_INFO) + + case tuple if tuple.productIterator.forall(numericWideningPrecedence.contains) => + val higherIndex = numericWideningPrecedence.lastIndexWhere(t => t == tp1 || t == tp2) + Some(numericWideningPrecedence(higherIndex)) + + case _ => None + } + } + + /** + * Test if we can do cast safely without lose of information. + */ + def canSafelyCast(from: TypeInformation[_], to: TypeInformation[_]): Boolean = (from, to) match { + case (_, STRING_TYPE_INFO) => true + + case tuple if tuple.productIterator.forall(numericWideningPrecedence.contains) => + if (numericWideningPrecedence.indexOf(from) < numericWideningPrecedence.indexOf(to)) { + true + } else { + false + } + + case _ => false + } + + /** + * All the supported cast types in flink-table. + * Note: This may lose information during the cast. + */ + def canCast(from: TypeInformation[_], to: TypeInformation[_]): Boolean = (from, to) match { + case (from, to) if from == to => true + + case (_, STRING_TYPE_INFO) => true + + case (_, DATE_TYPE_INFO) => false // Date type not supported yet. + case (_, VOID_TYPE_INFO) => false // Void type not supported + case (_, CHAR_TYPE_INFO) => false // Character type not supported. + + case (STRING_TYPE_INFO, _: NumericTypeInfo[_]) => true + case (STRING_TYPE_INFO, BOOLEAN_TYPE_INFO) => true + + case (BOOLEAN_TYPE_INFO, _: NumericTypeInfo[_]) => true + case (_: NumericTypeInfo[_], BOOLEAN_TYPE_INFO) => true + + case (_: NumericTypeInfo[_], _: NumericTypeInfo[_]) => true + + case _ => false + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableException.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ExprValidationResult.scala similarity index 56% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableException.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ExprValidationResult.scala index 3e298a4b75199..8571051a2597a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableException.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ExprValidationResult.scala @@ -15,9 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.table +package org.apache.flink.api.table.validate /** - * General Exception for all errors during table handling. - */ -class TableException(msg: String) extends RuntimeException(msg) + * Represents the result of `Expression.validateInput`. + */ +sealed trait ExprValidationResult { + def isFailure: Boolean = !isSuccess + def isSuccess: Boolean +} + +/** + * Represents the successful result of `Expression.checkInputDataTypes`. + */ +object ValidationSuccess extends ExprValidationResult { + val isSuccess: Boolean = true +} + +/** + * Represents the failing result of `Expression.checkInputDataTypes`, + * with a error message to show the reason of failure. + */ +case class ValidationFailure(message: String) extends ExprValidationResult { + val isSuccess: Boolean = false +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala new file mode 100644 index 0000000000000..726917e8a6683 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.validate + +import scala.collection.mutable +import scala.util.{Failure, Success, Try} + +import org.apache.flink.api.table.expressions._ +import org.apache.flink.api.table.ValidationException + +/** + * A catalog for looking up user defined functions, used during validation phase. + */ +class FunctionCatalog { + + private val functionBuilders = mutable.HashMap.empty[String, Class[_]] + + def registerFunction(name: String, builder: Class[_]): Unit = + functionBuilders.put(name.toLowerCase, builder) + + /** + * Lookup and create an expression if we find a match. + */ + def lookupFunction(name: String, children: Seq[Expression]): Expression = { + val funcClass = functionBuilders.get(name.toLowerCase).getOrElse { + throw new ValidationException(s"undefined function $name") + } + withChildren(funcClass, children) + } + + /** + * Instantiate a function using the provided `children`. + */ + private def withChildren(func: Class[_], children: Seq[Expression]): Expression = { + // Try to find a constructor accepts `Seq[Expression]` + Try(func.getDeclaredConstructor(classOf[Seq[_]])) match { + case Success(seqCtor) => + Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(e) => throw new ValidationException(e.getMessage) + } + case Failure(e) => + val childrenClass = Seq.fill(children.length)(classOf[Expression]) + // Try to find a constructor matching the exact number of children + Try(func.getDeclaredConstructor(childrenClass: _*)) match { + case Success(ctor) => + Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(e) => throw new ValidationException(e.getMessage) + } + case Failure(e) => + throw new ValidationException(s"Invalid number of arguments for function $func") + } + } + } + + /** + * Drop a function and return if the function existed. + */ + def dropFunction(name: String): Boolean = + functionBuilders.remove(name.toLowerCase).isDefined + + /** + * Drop all registered functions. + */ + def clear(): Unit = functionBuilders.clear() +} + +object FunctionCatalog { + + val buildInFunctions: Map[String, Class[_]] = Map( + // aggregate functions + "avg" -> classOf[Avg], + "count" -> classOf[Count], + "max" -> classOf[Max], + "min" -> classOf[Min], + "sum" -> classOf[Sum], + + // string functions + "charLength" -> classOf[CharLength], + "initCap" -> classOf[InitCap], + "like" -> classOf[Like], + "lowerCase" -> classOf[Lower], + "similar" -> classOf[Similar], + "subString" -> classOf[SubString], + "trim" -> classOf[Trim], + "upperCase" -> classOf[Upper], + + // math functions + "abs" -> classOf[Abs], + "ceil" -> classOf[Ceil], + "exp" -> classOf[Exp], + "floor" -> classOf[Floor], + "log10" -> classOf[Log10], + "ln" -> classOf[Ln], + "power" -> classOf[Power], + "mod" -> classOf[Mod] + ) + + /** + * Create a new function catalog with build-in functions. + */ + def withBuildIns: FunctionCatalog = { + val catalog = new FunctionCatalog() + buildInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) } + catalog + } +} diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java index 0a2cf57810497..8fdb2da27823b 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java @@ -24,10 +24,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.TableException; +import org.apache.flink.api.table.*; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.junit.Test; import org.junit.runner.RunWith; @@ -99,7 +96,7 @@ public void testRegisterExistingDatasetTable() throws Exception { tableEnv.registerDataSet("MyTable", ds2); } - @Test(expected = TableException.class) + @Test(expected = ValidationException.class) public void testScanUnregisteredTable() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -127,7 +124,7 @@ public void testTableRegister() throws Exception { compareResultAsText(results, expected); } - @Test(expected = TableException.class) + @Test(expected = ValidationException.class) public void testIllegalName() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -138,7 +135,7 @@ public void testIllegalName() throws Exception { tableEnv.registerTable("_DataSetTable_42", t); } - @Test(expected = TableException.class) + @Test(expected = ValidationException.class) public void testRegisterTableFromOtherEnv() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv1 = TableEnvironment.getTableEnvironment(env, config()); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java index d4af08d1edecf..ddf5884066cea 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java @@ -27,7 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.plan.PlanGenException; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; @@ -59,7 +59,7 @@ public void testAggregationTypes() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testAggregationOnNonExistingField() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -141,7 +141,7 @@ public void testAggregationWithTwoCount() throws Exception { compareResultAsText(results, expected); } - @Test(expected = PlanGenException.class) + @Test(expected = ValidationException.class) public void testNonWorkingDataTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -161,7 +161,7 @@ public void testNonWorkingDataTypes() throws Exception { compareResultAsText(results, expected); } - @Test(expected = UnsupportedOperationException.class) + @Test(expected = ValidationException.class) public void testNoNestedAggregation() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java index 02469045a3ba3..83c7cfd07a19b 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/ExpressionsITCase.java @@ -30,6 +30,8 @@ import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.codegen.CodeGenException; import static org.junit.Assert.fail; + +import org.apache.flink.api.table.ValidationException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -154,7 +156,7 @@ public void testEval() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testEvalInvalidTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java index 284834781d006..7a2bedf116362 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FilterITCase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.table.TableEnvironment; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.junit.Test; import org.junit.runner.RunWith; @@ -152,7 +153,7 @@ public void testIntegerBiggerThan128() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testFilterInvalidField() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java index 70dd79317a0ad..19060404e3510 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/GroupedAggregationsITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.table.TableEnvironment; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; @@ -40,7 +41,7 @@ public GroupedAggregationsITCase(TestExecutionMode mode){ super(mode); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testGroupingOnNonExistentField() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -54,7 +55,7 @@ public void testGroupingOnNonExistentField() throws Exception { .select("a.avg"); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testGroupingInvalidSelection() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java index 4c40596985bfd..e6db3b0010146 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java @@ -26,7 +26,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.TableException; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; @@ -121,7 +121,7 @@ public void testJoinWithMultipleKeys() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testJoinNonExistingKey() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -136,7 +136,7 @@ public void testJoinNonExistingKey() throws Exception { in1.join(in2).where("foo === e").select("c, g"); } - @Test(expected = TableException.class) + @Test(expected = ValidationException.class) public void testJoinWithNonMatchingKeyTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -154,7 +154,7 @@ public void testJoinWithNonMatchingKeyTypes() throws Exception { tableEnv.toDataSet(result, Row.class).collect(); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testJoinWithAmbiguousFields() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -189,7 +189,7 @@ public void testJoinWithAggregation() throws Exception { compareResultAsText(results, expected); } - @Test(expected = TableException.class) + @Test(expected = ValidationException.class) public void testJoinTablesFromDifferentEnvs() { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv1 = TableEnvironment.getTableEnvironment(env); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java index 7c9478ac40516..e48914c132747 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/SelectITCase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.table.TableEnvironment; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.junit.Test; import org.junit.runner.RunWith; @@ -103,7 +104,7 @@ public void testSimpleSelectRenameAll() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testSelectInvalidField() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -115,7 +116,7 @@ public void testSelectInvalidField() throws Exception { .select("a + 1, foo + 2"); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testSelectAmbiguousFieldNames() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/StringExpressionsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/StringExpressionsITCase.java index e55bd227fce81..db5eac9bc7ec3 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/StringExpressionsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/StringExpressionsITCase.java @@ -21,10 +21,10 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.table.TableEnvironment; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.api.table.Row; import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.codegen.CodeGenException; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.table.BatchTableEnvironment; @@ -63,6 +63,26 @@ public void testSubstring() throws Exception { compareResultAsText(results, expected); } + @Test + public void testSubstringWithByteStart() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + + DataSet> ds = env.fromElements( + new Tuple2<>("AAAA", (byte) 2), + new Tuple2<>("BBBB", (byte) 1)); + + Table in = tableEnv.fromDataSet(ds, "a, b"); + + Table result = in + .select("a.substring(1, b)"); + + DataSet resultSet = tableEnv.toDataSet(result, Row.class); + List results = resultSet.collect(); + String expected = "AA\nB"; + compareResultAsText(results, expected); + } + @Test public void testSubstringWithMaxEnd() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -83,7 +103,7 @@ public void testSubstringWithMaxEnd() throws Exception { compareResultAsText(results, expected); } - @Test(expected = CodeGenException.class) + @Test(expected = ValidationException.class) public void testNonWorkingSubstring1() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -102,7 +122,7 @@ public void testNonWorkingSubstring1() throws Exception { resultSet.collect(); } - @Test(expected = CodeGenException.class) + @Test(expected = ValidationException.class) public void testNonWorkingSubstring2() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -121,7 +141,7 @@ public void testNonWorkingSubstring2() throws Exception { resultSet.collect(); } - @Test(expected = CodeGenException.class) + @Test(expected = ValidationException.class) public void testGeneratedCodeForStringComparison() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -132,7 +152,7 @@ public void testGeneratedCodeForStringComparison() throws Exception { DataSet resultSet = tableEnv.toDataSet(res, Row.class); } - @Test(expected = CodeGenException.class) + @Test(expected = ValidationException.class) public void testGeneratedCodeForIntegerEqualsComparison() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -143,7 +163,7 @@ public void testGeneratedCodeForIntegerEqualsComparison() throws Exception { DataSet resultSet = tableEnv.toDataSet(res, Row.class); } - @Test(expected = CodeGenException.class) + @Test(expected = ValidationException.class) public void testGeneratedCodeForIntegerGreaterComparison() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java index a7805f8fd6e23..853cd7f5403ce 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java @@ -26,7 +26,7 @@ import org.apache.flink.api.table.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.TableException; +import org.apache.flink.api.table.ValidationException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.junit.Test; @@ -80,7 +80,7 @@ public void testUnionWithFilter() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testUnionIncompatibleNumberOfFields() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -95,7 +95,7 @@ public void testUnionIncompatibleNumberOfFields() throws Exception { in1.unionAll(in2); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testUnionIncompatibleFieldsName() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -110,7 +110,7 @@ public void testUnionIncompatibleFieldsName() throws Exception { in1.unionAll(in2); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = ValidationException.class) public void testUnionIncompatibleFieldTypes() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); @@ -168,7 +168,7 @@ public void testUnionWithJoin() throws Exception { compareResultAsText(results, expected); } - @Test(expected = TableException.class) + @Test(expected = ValidationException.class) public void testUnionTablesFromDifferentEnvs() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tEnv1 = TableEnvironment.getTableEnvironment(env); diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala index 80a491bdda89c..c33e1ef925a42 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala @@ -23,7 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -89,7 +89,7 @@ class TableEnvironmentITCase( tEnv.registerDataSet("MyTable", ds2) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testScanUnregisteredTable(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -130,7 +130,7 @@ class TableEnvironmentITCase( tEnv.registerDataSet("MyTable", t2) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testRegisterTableFromOtherEnv(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv1 = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala index c34edbc0a153c..6c413e5f8565e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala @@ -21,8 +21,7 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.examples.scala.WordCountTable.{WC => MyWC} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} @@ -49,7 +48,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testAggregationOnNonExistingField(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -137,7 +136,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa TestBaseUtils.compareResultAsText(result.asJava, expected) } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[ValidationException]) def testNonWorkingAggregationDataTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -150,7 +149,7 @@ class AggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa t.collect() } - @Test(expected = classOf[UnsupportedOperationException]) + @Test(expected = classOf[ValidationException]) def testNoNestedAggregations(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala index fdbe3c9aa1107..9a0a035d01a2a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ExpressionsITCase.scala @@ -27,7 +27,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.codegen.CodeGenException import org.apache.flink.api.table.expressions.{Literal, Null} -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit.Assert._ @@ -144,7 +144,7 @@ class ExpressionsITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testEvalInvalidTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala index 5deff9e864358..ee0356f3a8957 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/FilterITCase.scala @@ -24,7 +24,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -174,7 +174,7 @@ class FilterITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testFilterInvalidFieldName(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala index cbd42b21a8ecf..8889b3766c155 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/GroupedAggregationsITCase.scala @@ -21,7 +21,7 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} import org.junit._ @@ -33,7 +33,7 @@ import scala.collection.JavaConverters._ @RunWith(classOf[Parameterized]) class GroupedAggregationsITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) { - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testGroupingOnNonExistentField(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -45,7 +45,7 @@ class GroupedAggregationsITCase(mode: TestExecutionMode) extends MultipleProgram .select('a.avg) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testGroupingInvalidSelection(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala index 3a89de2a4d5b2..1a9be93c330e5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} import org.junit._ @@ -97,7 +97,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testJoinNonExistingKey(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -111,7 +111,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .select('c, 'g) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testJoinWithNonMatchingKeyTypes(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -125,7 +125,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .select('c, 'g).collect() } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testJoinWithAmbiguousFields(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -258,7 +258,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testJoinTablesFromDifferentEnvs(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv1 = TableEnvironment.getTableEnvironment(env) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala index c6a61225d32e7..9aed5a7a3c6e2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SelectITCase.scala @@ -23,7 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -104,7 +104,7 @@ class SelectITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testSelectInvalidFieldFields(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -114,7 +114,7 @@ class SelectITCase( .select('a, 'foo) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testSelectAmbiguousRenaming(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -124,7 +124,7 @@ class SelectITCase( .select('a + 1 as 'foo, 'b + 2 as 'foo).toDataSet[Row].print() } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testSelectAmbiguousRenaming2(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/StringExpressionsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/StringExpressionsITCase.scala index e42830649bb4e..f7d998b38d871 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/StringExpressionsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/StringExpressionsITCase.scala @@ -20,8 +20,7 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.codegen.CodeGenException -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} import org.junit._ @@ -59,7 +58,7 @@ class StringExpressionsITCase(mode: TestExecutionMode) extends MultipleProgramsT TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[CodeGenException]) + @Test(expected = classOf[ValidationException]) def testNonWorkingSubstring1(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -71,7 +70,7 @@ class StringExpressionsITCase(mode: TestExecutionMode) extends MultipleProgramsT t.toDataSet[Row].collect() } - @Test(expected = classOf[CodeGenException]) + @Test(expected = classOf[ValidationException]) def testNonWorkingSubstring2(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala index 7c109a3abae53..29427a5d29caa 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala @@ -23,7 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -90,7 +90,7 @@ class UnionITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testUnionDifferentFieldNames(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -102,7 +102,7 @@ class UnionITCase( ds1.unionAll(ds2) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testUnionDifferentFieldTypes(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -159,7 +159,7 @@ class UnionITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testUnionTablesFromDifferentEnvs(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv1 = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala index 56daa6bf76b67..fe606e0e0e23a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala @@ -30,7 +30,7 @@ import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment} import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedFunction} import org.apache.flink.api.table.expressions.Expression import org.apache.flink.api.table.runtime.FunctionCompiler -import org.apache.flink.api.table.{TableConfig, TableEnvironment} +import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig, TableEnvironment} import org.mockito.Mockito._ /** @@ -82,8 +82,12 @@ object ExpressionEvaluator { } def evaluate(data: Any, typeInfo: TypeInformation[Any], expr: Expression): String = { - val relBuilder = prepareTable(typeInfo)._2 - evaluate(data, typeInfo, relBuilder, expr.toRexNode(relBuilder)) + val table = prepareTable(typeInfo) + val env = table._3 + val resolvedExpr = + env.asInstanceOf[BatchTableEnvironment].scan("myTable").select(expr). + getRelNode.asInstanceOf[LogicalProject].getChildExps.get(0) + evaluate(data, typeInfo, table._2, resolvedExpr) } def evaluate( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala index ff0e961856a37..feda75fb15fd4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala @@ -19,9 +19,9 @@ package org.apache.flink.api.scala.stream.table import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.stream.utils.{StreamTestData, StreamITCase} +import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData} import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.junit.Assert._ @@ -70,7 +70,7 @@ class UnionITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testUnionFieldsNameNotOverlap1(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -88,7 +88,7 @@ class UnionITCase extends StreamingMultipleProgramsTestBase { assertEquals(true, StreamITCase.testResults.isEmpty) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[ValidationException]) def testUnionFieldsNameNotOverlap2(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -108,7 +108,7 @@ class UnionITCase extends StreamingMultipleProgramsTestBase { assertEquals(true, StreamITCase.testResults.isEmpty) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testUnionTablesFromDifferentEnvs(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv1 = TableEnvironment.getTableEnvironment(env) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala index e6309a28f963b..a382447bcb9d0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala @@ -49,6 +49,13 @@ class UnsupportedOpsTest extends StreamingMultipleProgramsTestBase { StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).distinct() } + @Test(expected = classOf[TableException]) + def testSort(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).orderBy('_1.desc) + } + @Test(expected = classOf[TableException]) def testJoin(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment From 6653e0af59802e0564ebbbb4de784e44f2f39f13 Mon Sep 17 00:00:00 2001 From: Gordon Tai Date: Sat, 16 Apr 2016 11:33:17 +0800 Subject: [PATCH 29/70] [FLINK-3229] Initial working version for FlinkKinesisConsumer. This closes #1911 --- docs/apis/streaming/connectors/kinesis.md | 38 ++ .../flink-connector-kinesis/pom.xml | 74 ++- .../kinesis/FlinkKinesisConsumer.java | 493 +++++++++++++++++ .../kinesis/FlinkKinesisProducer.java | 1 + .../config/CredentialProviderType.java | 39 ++ .../kinesis/config/InitialPosition.java | 33 ++ .../config/KinesisConfigConstants.java | 71 +++ .../kinesis/examples/ConsumeFromKinesis.java | 54 ++ .../kinesis/examples/ProduceIntoKinesis.java | 4 - .../kinesis/internals/KinesisDataFetcher.java | 171 ++++++ .../internals/ShardConsumerThread.java | 151 ++++++ .../kinesis/model/KinesisStreamShard.java | 135 +++++ .../kinesis/model/SentinelSequenceNumber.java | 45 ++ .../kinesis/proxy/KinesisProxy.java | 209 ++++++++ .../KinesisDeserializationSchema.java | 55 ++ .../KinesisDeserializationSchemaWrapper.java | 53 ++ .../KinesisSerializationSchema.java | 2 +- .../connectors/kinesis/util/AWSUtil.java | 99 ++++ .../kinesis/FlinkKinesisConsumerTest.java | 507 ++++++++++++++++++ .../internals/KinesisDataFetcherTest.java | 48 ++ .../internals/ShardConsumerThreadTest.java | 207 +++++++ .../ManualConsumerProducerTest.java | 115 ++++ .../manualtests/ManualExactlyOnceTest.java | 277 ++++++++++ .../ManualProducerTest.java} | 11 +- .../testutils/KinesisShardIdGenerator.java | 24 + .../ReferenceKinesisShardTopologies.java | 117 ++++ .../TestableFlinkKinesisConsumer.java | 58 ++ 27 files changed, 3080 insertions(+), 11 deletions(-) create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/CredentialProviderType.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/InitialPosition.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java rename flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/{ => serialization}/KinesisSerializationSchema.java (95%) create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java rename flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/{ManualTest.java => manualtests/ManualProducerTest.java} (85%) create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java diff --git a/docs/apis/streaming/connectors/kinesis.md b/docs/apis/streaming/connectors/kinesis.md index 52ba6b0ecf506..91a19968502cb 100644 --- a/docs/apis/streaming/connectors/kinesis.md +++ b/docs/apis/streaming/connectors/kinesis.md @@ -44,6 +44,44 @@ Linking to the flink-connector-kinesis will include ASL licensed code into your Note that the streaming connectors are not part of the binary distribution. See linking with them for cluster execution [here]({{site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution). +#### Usage of Consumer + +The `FlinkKinesisConsumer` can be used to pull data from multiple Kinesis streams within the same AWS region in parallel. +It participates with Flink's distributed snapshot checkpointing and provides exactly-once processing guarantees. Note +that the current version can not handle resharding of Kinesis streams. When Kinesis streams are resharded, the consumer +will fail and the Flink streaming job must be resubmitted. + +Before consuming data from Kinesis streams, make sure that all streams are created with the status "ACTIVE" in the AWS dashboard. + +
+
+{% highlight java %} +Properties kinesisConsumerConfig = new Properties(); +kinesisConsumerConfig.put(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); +kinesisConsumerConfig.put(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE, "BASIC"); +kinesisConsumerConfig.put( + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, + "aws_access_key_id_here"); +kinesisConsumerConfig.put( + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, + "aws_secret_key_here"); +kinesisConsumerConfig.put(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "LATEST"); + +StreamExecutionEnvironment env = StreamExecutionEnvironment.getEnvironment(); + +DataStream kinesisRecords = env.addSource(new FlinkKinesisConsumer<>( + "kinesis_stream_name", new SimpleStringSchema(), kinesisConsumerConfig)) +{% endhighlight %} +
+
+ +The above is a simple example of using the consumer. Configuration for the consumer is supplied with a `java.util.Properties` +instance, with which the configuration setting keys used can be found in `KinesisConfigConstants`. The example +demonstrates consuming a single Kinesis stream in the AWS region "us-east-1". The AWS credentials is supplied using the basic method in which +the AWS access key ID and secret key are directly supplied in the configuration (other options are setting +`KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE` to `ENV_VAR`, `SYS_PROP`, and `PROFILE`). Also, data is being consumed +from the newest position in the Kinesis stream (the other option will be setting `KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE` +to `TRIM_HORIZON`, which lets the consumer start reading the Kinesis stream from the earliest record possible). #### Usage of Producer diff --git a/flink-streaming-connectors/flink-connector-kinesis/pom.xml b/flink-streaming-connectors/flink-connector-kinesis/pom.xml index d23495abd499e..ce49fd7d95ab5 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/pom.xml +++ b/flink-streaming-connectors/flink-connector-kinesis/pom.xml @@ -33,26 +33,94 @@ under the License. flink-connector-kinesis_2.10 flink-connector-kinesis - 0.10.2 + 1.10.71 + 1.6.2 + 0.10.2 jar + org.apache.flink flink-streaming-java_2.10 ${project.version} + provided + + + + com.google.guava + guava + ${guava.version} + + + + org.apache.flink + flink-tests_2.10 + ${project.version} + test + test-jar + + + + org.apache.flink + flink-test-utils_2.10 + ${project.version} + test + + + com.amazonaws + aws-java-sdk-kinesis + ${aws.sdk.version} + + com.amazonaws amazon-kinesis-producer - ${kinesis-producer.version} + ${aws.kinesis-kpl.version} + + + + com.amazonaws + amazon-kinesis-client + ${aws.kinesis-kcl.version} + + + + com.amazonaws + aws-java-sdk-dynamodb + + + com.amazonaws + aws-java-sdk-cloudwatch + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java new file mode 100644 index 0000000000000..97cd048583407 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java @@ -0,0 +1,493 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis; + +import com.amazonaws.regions.Regions; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; +import org.apache.flink.streaming.connectors.kinesis.config.CredentialProviderType; +import org.apache.flink.streaming.connectors.kinesis.config.InitialPosition; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; +import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; +import java.util.Properties; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * The Flink Kinesis Consumer is a parallel streaming data source that pulls data from multiple AWS Kinesis streams + * within the same AWS service region. Each instance of the consumer is responsible for fetching data records from + * one or more Kinesis shards. + * + *

To leverage Flink's checkpointing mechanics for exactly-once streaming processing guarantees, the Flink Kinesis + * consumer is implemented with the AWS Java SDK, instead of the officially recommended AWS Kinesis Client Library, for + * low-level control on the management of stream state. The Flink Kinesis Connector also supports setting the initial + * starting points of Kinesis streams, namely TRIM_HORIZON and LATEST.

+ * + *

NOTE: The current implementation does not correctly handle resharding of AWS Kinesis streams.

+ *

NOTE: Since Kinesis and Kafka share many common abstractions, the implementation is heavily based on + * the Flink Kafka Consumer.

+ * + * @param the type of data emitted + */ +public class FlinkKinesisConsumer extends RichParallelSourceFunction + implements CheckpointedAsynchronously>, ResultTypeQueryable { + + private static final long serialVersionUID = 4724006128720664870L; + + private static final Logger LOG = LoggerFactory.getLogger(FlinkKinesisConsumer.class); + + // ------------------------------------------------------------------------ + // Consumer properties + // ------------------------------------------------------------------------ + + /** The complete list of shards */ + private final List shards; + + /** Properties to parametrize settings such as AWS service region, initial position in stream, + * shard list retrieval behaviours, etc */ + private final Properties configProps; + + /** User supplied deseriliazation schema to convert Kinesis byte messages to Flink objects */ + private final KinesisDeserializationSchema deserializer; + + // ------------------------------------------------------------------------ + // Runtime state + // ------------------------------------------------------------------------ + + /** Per-task fetcher for Kinesis data records, where each fetcher pulls data from one or more Kinesis shards */ + private transient KinesisDataFetcher fetcher; + + /** The sequence numbers of the last fetched data records from Kinesis by this task */ + private transient HashMap lastSequenceNums; + + /** The sequence numbers to restore to upon restore from failure */ + private transient HashMap sequenceNumsToRestore; + + private volatile boolean hasAssignedShards; + + private volatile boolean running = true; + + + // ------------------------------------------------------------------------ + // Constructors + // ------------------------------------------------------------------------ + + /** + * Creates a new Flink Kinesis Consumer. + * + *

The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming + * from are configured with a {@link Properties} instance.

+ * + * @param stream + * The single AWS Kinesis stream to read from. + * @param deserializer + * The deserializer used to convert raw bytes of Kinesis records to Java objects (without key). + * @param configProps + * The properties used to configure AWS credentials, AWS region, and initial starting position. + */ + public FlinkKinesisConsumer(String stream, DeserializationSchema deserializer, Properties configProps) { + this(stream, new KinesisDeserializationSchemaWrapper(deserializer), configProps); + } + + /** + * Creates a new Flink Kinesis Consumer. + * + *

The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming + * from are configured with a {@link Properties} instance.

+ * + * @param stream + * The single AWS Kinesis stream to read from. + * @param deserializer + * The keyed deserializer used to convert raw bytes of Kinesis records to Java objects. + * @param configProps + * The properties used to configure AWS credentials, AWS region, and initial starting position. + */ + public FlinkKinesisConsumer(String stream, KinesisDeserializationSchema deserializer, Properties configProps) { + this(Collections.singletonList(stream), deserializer, configProps); + } + + /** + * Creates a new Flink Kinesis Consumer. + * + *

The AWS credentials to be used, AWS region of the Kinesis streams, initial position to start streaming + * from are configured with a {@link Properties} instance.

+ * + * @param streams + * The AWS Kinesis streams to read from. + * @param deserializer + * The keyed deserializer used to convert raw bytes of Kinesis records to Java objects. + * @param configProps + * The properties used to configure AWS credentials, AWS region, and initial starting position. + */ + public FlinkKinesisConsumer(List streams, KinesisDeserializationSchema deserializer, Properties configProps) { + checkNotNull(streams, "streams can not be null"); + + this.configProps = checkNotNull(configProps, "configProps can not be null"); + + // check the configuration properties for any conflicting settings + validatePropertiesConfig(this.configProps); + + this.deserializer = checkNotNull(deserializer, "deserializer can not be null"); + + this.shards = new KinesisProxy(configProps).getShardList(streams); + if (shards.size() == 0) { + throw new RuntimeException("Unable to retrieve any shards for the requested streams " + streams.toString() + "."); + } + + if (LOG.isInfoEnabled()) { + Map shardCountPerStream = new HashMap<>(); + for (KinesisStreamShard shard : shards) { + Integer shardCount = shardCountPerStream.get(shard.getStreamName()); + if (shardCount == null) { + shardCount = 1; + } else { + shardCount++; + } + shardCountPerStream.put(shard.getStreamName(), shardCount); + } + StringBuilder sb = new StringBuilder(); + for (Map.Entry streamAndShardCountPair : shardCountPerStream.entrySet()) { + sb.append(streamAndShardCountPair.getKey()).append(" (").append(streamAndShardCountPair.getValue()).append("), "); + } + LOG.info("Flink Kinesis Consumer is going to read the following streams (with number of shards): {}", sb.toString()); + } + } + + // ------------------------------------------------------------------------ + // Source life cycle + // ------------------------------------------------------------------------ + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + final int numFlinkConsumerTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + final int thisConsumerTaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + + // pick which shards this consumer task works on, in a round-robin fashion + List assignedShards = assignShards(this.shards, numFlinkConsumerTasks, thisConsumerTaskIndex); + + // if there are no shards assigned to this consumer task, return without doing anything. + if (assignedShards.isEmpty()) { + LOG.info("Consumer task {} has no shards assigned to it", thisConsumerTaskIndex); + hasAssignedShards = false; + return; + } else { + hasAssignedShards = true; + } + + if (LOG.isInfoEnabled()) { + StringBuilder sb = new StringBuilder(); + for (KinesisStreamShard shard : assignedShards) { + sb.append(shard.getStreamName()).append(":").append(shard.getShardId()).append(", "); + } + LOG.info("Consumer task {} will read shards {} out of a total of {} shards", + thisConsumerTaskIndex, sb.toString(), this.shards.size()); + } + + fetcher = new KinesisDataFetcher(assignedShards, configProps, getRuntimeContext().getTaskName()); + + // restore to the last known sequence numbers from the latest complete snapshot + if (sequenceNumsToRestore != null) { + if (LOG.isInfoEnabled()) { + LOG.info("Consumer task {} is restoring sequence numbers from previous checkpointed state", thisConsumerTaskIndex); + } + + for (Map.Entry restoreSequenceNum : sequenceNumsToRestore.entrySet()) { + // advance the corresponding shard to the last known sequence number + fetcher.advanceSequenceNumberTo(restoreSequenceNum.getKey(), restoreSequenceNum.getValue()); + } + + if (LOG.isInfoEnabled()) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry restoreSequenceNo : sequenceNumsToRestore.entrySet()) { + KinesisStreamShard shard = restoreSequenceNo.getKey(); + sb.append(shard.getStreamName()).append(":").append(shard.getShardId()) + .append(" -> ").append(restoreSequenceNo.getValue()).append(", "); + } + LOG.info("Advanced the starting sequence numbers of consumer task {}: {}", thisConsumerTaskIndex, sb.toString()); + } + + // initialize sequence numbers with restored state + lastSequenceNums = sequenceNumsToRestore; + sequenceNumsToRestore = null; + } else { + // start fresh with empty sequence numbers if there are no snapshots to restore from. + lastSequenceNums = new HashMap<>(); + + // advance all assigned shards of this consumer task to either the earliest or latest sequence number, + // depending on the properties configuration (default is to set to latest sequence number). + InitialPosition initialPosition = InitialPosition.valueOf(configProps.getProperty( + KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, InitialPosition.LATEST.toString())); + + SentinelSequenceNumber sentinelSequenceNum; + switch (initialPosition) { + case TRIM_HORIZON: + sentinelSequenceNum = SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM; + break; + case LATEST: + default: + sentinelSequenceNum = SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM; + } + + for (KinesisStreamShard assignedShard : assignedShards) { + fetcher.advanceSequenceNumberTo(assignedShard, sentinelSequenceNum.toString()); + } + + if (LOG.isInfoEnabled()) { + StringBuilder sb = new StringBuilder(); + for (KinesisStreamShard assignedShard : assignedShards) { + sb.append(assignedShard.getStreamName()).append(":").append(assignedShard.getShardId()) + .append(" -> ").append(sentinelSequenceNum.toString()).append(", "); + } + LOG.info("Advanced the starting sequence numbers of consumer task {}: {}", thisConsumerTaskIndex, sb.toString()); + } + } + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + if (hasAssignedShards) { + fetcher.run(sourceContext, deserializer, lastSequenceNums); + } else { + // this source never completes because there is no assigned shards, + // so emit a Long.MAX_VALUE watermark to no block watermark forwarding + sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); + + final Object waitLock = new Object(); + while (running) { + try { + synchronized (waitLock) { + waitLock.wait(); + } + } catch (InterruptedException e) { + // do nothing + } + } + } + + sourceContext.close(); + } + + @Override + public void cancel() { + running = false; + + // interrupt the fetcher of any work + KinesisDataFetcher fetcher = this.fetcher; + this.fetcher = null; + if (fetcher != null) { + try { + fetcher.close(); + } catch (IOException e) { + LOG.warn("Error while closing Kinesis data fetcher", e); + } + } + } + + @Override + public void close() throws Exception { + cancel(); + super.close(); + } + + @Override + public TypeInformation getProducedType() { + return deserializer.getProducedType(); + } + + // ------------------------------------------------------------------------ + // State Snapshot & Restore + // ------------------------------------------------------------------------ + + @Override + public HashMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + if (lastSequenceNums == null) { + LOG.debug("snapshotState() requested on not yet opened source; returning null."); + return null; + } + + if (!running) { + LOG.debug("snapshotState() called on closed source; returning null."); + return null; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Snapshotting state. ..."); + } + + @SuppressWarnings("unchecked") + HashMap currentSequenceNums = (HashMap) lastSequenceNums.clone(); + return currentSequenceNums; + } + + @Override + public void restoreState(HashMap restoredState) throws Exception { + sequenceNumsToRestore = restoredState; + } + + // ------------------------------------------------------------------------ + // Miscellaneous utilities + // ------------------------------------------------------------------------ + + /** + * Utility function to assign shards to a specific consumer task in a round-robin fashion. + */ + protected static List assignShards(List shards, int numFlinkConsumerTasks, int thisConsumerTaskIndex) { + checkArgument(numFlinkConsumerTasks > 0); + checkArgument(thisConsumerTaskIndex < numFlinkConsumerTasks); + + List closedShards = new ArrayList<>(); + List openShards = new ArrayList<>(); + + for (KinesisStreamShard shard : shards) { + if (shard.isClosed()) { + closedShards.add(shard); + } else { + openShards.add(shard); + } + } + + List subscribedShards = new ArrayList<>(); + + // separately round-robin assign open and closed shards so that all tasks have a fair chance of being + // assigned open shards (set of data records in closed shards are bounded) + + for (int i = 0; i < closedShards.size(); i++) { + if (i % numFlinkConsumerTasks == thisConsumerTaskIndex) { + subscribedShards.add(closedShards.get(i)); + } + } + + for (int i = 0; i < openShards.size(); i++) { + if (i % numFlinkConsumerTasks == thisConsumerTaskIndex) { + subscribedShards.add(openShards.get(i)); + } + } + return subscribedShards; + } + + /** + * Checks that the values specified for config keys in the properties config is recognizable. + */ + protected static void validatePropertiesConfig(Properties config) { + if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE)) { + // if the credential provider type is not specified, it will default to BASIC later on, + // so the Access Key ID and Secret Key must be given + if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID) + || !config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY)) { + throw new IllegalArgumentException("Need to set values for AWS Access Key ID and Secret Key when using the BASIC AWS credential provider type."); + } + } else { + String credentialsProviderType = config.getProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE); + + // value specified for KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE needs to be recognizable + CredentialProviderType providerType; + try { + providerType = CredentialProviderType.valueOf(credentialsProviderType); + } catch (IllegalArgumentException e) { + StringBuilder sb = new StringBuilder(); + for (CredentialProviderType type : CredentialProviderType.values()) { + sb.append(type.toString()).append(", "); + } + throw new IllegalArgumentException("Invalid AWS Credential Provider Type set in config. Valid values are: " + sb.toString()); + } + + // if BASIC type is used, also check that the Access Key ID and Secret Key is supplied + if (providerType == CredentialProviderType.BASIC) { + if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID) + || !config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY)) { + throw new IllegalArgumentException("Need to set values for AWS Access Key ID and Secret Key when using the BASIC AWS credential provider type."); + } + } + } + + if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_REGION)) { + throw new IllegalArgumentException("The AWS region must be set in the config."); + } else { + // specified AWS Region name must be recognizable + if (!AWSUtil.isValidRegion(config.getProperty(KinesisConfigConstants.CONFIG_AWS_REGION))) { + StringBuilder sb = new StringBuilder(); + for (Regions region : Regions.values()) { + sb.append(region.getName()).append(", "); + } + throw new IllegalArgumentException("Invalid AWS region set in config. Valid values are: " + sb.toString()); + } + } + + if (config.containsKey(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE)) { + String initPosType = config.getProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE); + + // specified initial position in stream must be either LATEST or TRIM_HORIZON + try { + InitialPosition.valueOf(initPosType); + } catch (IllegalArgumentException e) { + StringBuilder sb = new StringBuilder(); + for (InitialPosition pos : InitialPosition.values()) { + sb.append(pos.toString()).append(", "); + } + throw new IllegalArgumentException("Invalid initial position in stream set in config. Valid values are: " + sb.toString()); + } + } + + if (config.containsKey(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES)) { + try { + Integer.parseInt(config.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES)); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid value given for describeStream stream operation retry count. Must be a valid integer value."); + } + } + + if (config.containsKey(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF)) { + try { + Long.parseLong(config.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF)); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid value given for describeStream stream operation backoff milliseconds. Must be a valid long value."); + } + } + + if (config.containsKey(KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET)) { + try { + Integer.parseInt(config.getProperty(KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET)); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid value given for maximum records per getRecords shard operation. Must be a valid integer value."); + } + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java index 4da56e48955eb..9c9bd280fa8ee 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/CredentialProviderType.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/CredentialProviderType.java new file mode 100644 index 0000000000000..d0561281c590e --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/CredentialProviderType.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.config; + +import com.amazonaws.auth.AWSCredentialsProvider; + +/** + * Possible configuration values for the type of credential provider to use when accessing AWS Kinesis. + * Internally, a corresponding implementation of {@link AWSCredentialsProvider} will be used. + */ +public enum CredentialProviderType { + + /** Look for the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY to create AWS credentials */ + ENV_VAR, + + /** Look for Java system properties aws.accessKeyId and aws.secretKey to create AWS credentials */ + SYS_PROP, + + /** Use a AWS credentials profile file to create the AWS credentials */ + PROFILE, + + /** Simply create AWS credentials by supplying the AWS access key ID and AWS secret key in the configuration properties */ + BASIC +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/InitialPosition.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/InitialPosition.java new file mode 100644 index 0000000000000..1b3cd10e26354 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/InitialPosition.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.config; + +import com.amazonaws.services.kinesis.model.ShardIteratorType; + +/** + * The initial position to start reading shards from. This will affect the {@link ShardIteratorType} used + * when the consumer tasks retrieve the first shard iterator for each Kinesis shard. + */ +public enum InitialPosition { + + /** Start reading from the earliest possible record in the stream (excluding expired data records) */ + TRIM_HORIZON, + + /** Start reading from the latest incoming record */ + LATEST +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java new file mode 100644 index 0000000000000..ce394c70bbe35 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.config; + +/** + * Keys and default values used to configure the Kinesis consumer. + */ +public class KinesisConfigConstants { + + // ------------------------------------------------------------------------ + // Configuration Keys + // ------------------------------------------------------------------------ + + /** The max retries to retrieve metadata from a Kinesis stream using describeStream API + * (Note: describeStream attempts may be temporarily blocked due to AWS capping 5 attempts per sec) */ + public static final String CONFIG_STREAM_DESCRIBE_RETRIES = "flink.stream.describe.retry"; + + /** The backoff time between each describeStream attempt */ + public static final String CONFIG_STREAM_DESCRIBE_BACKOFF = "flink.stream.describe.backoff"; + + /** The maximum number of records to try to get each time we fetch records from a AWS Kinesis shard */ + public static final String CONFIG_SHARD_RECORDS_PER_GET = "flink.shard.getrecords.maxcount"; + + /** The initial position to start reading Kinesis streams from (LATEST is used if not set) */ + public static final String CONFIG_STREAM_INIT_POSITION_TYPE = "flink.stream.initpos.type"; + + /** The credential provider type to use when AWS credentials are required (BASIC is used if not set)*/ + public static final String CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE = "aws.credentials.provider"; + + /** The AWS access key ID to use when setting credentials provider type to BASIC */ + public static final String CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID = "aws.credentials.provider.basic.accesskeyid"; + + /** The AWS secret key to use when setting credentials provider type to BASIC */ + public static final String CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY = "aws.credentials.provider.basic.secretkey"; + + /** Optional configuration for profile path if credential provider type is set to be PROFILE */ + public static final String CONFIG_AWS_CREDENTIALS_PROVIDER_PROFILE_PATH = "aws.credentials.provider.profile.path"; + + /** Optional configuration for profile name if credential provider type is set to be PROFILE */ + public static final String CONFIG_AWS_CREDENTIALS_PROVIDER_PROFILE_NAME = "aws.credentials.provider.profile.name"; + + /** The AWS region of the Kinesis streams to be pulled ("us-east-1" is used if not set) */ + public static final String CONFIG_AWS_REGION = "aws.region"; + + + // ------------------------------------------------------------------------ + // Default configuration values + // ------------------------------------------------------------------------ + + public static final int DEFAULT_STREAM_DESCRIBE_RETRY_TIMES = 3; + + public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF = 1000L; + + public static final int DEFAULT_SHARD_RECORDS_PER_GET = 100; + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java new file mode 100644 index 0000000000000..8bae66587e23f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromKinesis.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kinesis.examples; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; + +import java.util.Properties; + +/** + * This is an example on how to consume data from Kinesis + */ +public class ConsumeFromKinesis { + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.setParallelism(1); + + Properties kinesisConsumerConfig = new Properties(); + kinesisConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, pt.getRequired("region")); + kinesisConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accesskey")); + kinesisConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretkey")); + + DataStream kinesis = see.addSource(new FlinkKinesisConsumer<>( + "flink-test", + new SimpleStringSchema(), + kinesisConsumerConfig)); + + kinesis.print(); + + see.execute(); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java index fdee08115a95d..57727a0a8b135 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java @@ -22,12 +22,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; -import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner; -import org.apache.flink.streaming.connectors.kinesis.KinesisSerializationSchema; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import java.nio.ByteBuffer; - /** * This is an example on how to produce data into Kinesis */ diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java new file mode 100644 index 0000000000000..9bf8468e689d7 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.internals; + +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.HashMap; +import java.util.List; +import java.util.ArrayList; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards. + * The fetcher spawns a single thread for connection to each shard. + */ +public class KinesisDataFetcher { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class); + + /** Config properties for the Flink Kinesis Consumer */ + private final Properties configProps; + + /** The name of the consumer task that this fetcher was instantiated */ + private final String taskName; + + /** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */ + private HashMap assignedShardsWithStartingSequenceNum; + + /** Reference to the thread that executed run() */ + private volatile Thread mainThread; + + /** Reference to the first error thrown by any of the spawned shard connection threads */ + private final AtomicReference error; + + private volatile boolean running = true; + + /** + * Creates a new Kinesis Data Fetcher for the specified set of shards + * + * @param assignedShards the shards that this fetcher will pull data from + * @param configProps the configuration properties of this Flink Kinesis Consumer + * @param taskName the task name of this consumer task + */ + public KinesisDataFetcher(List assignedShards, Properties configProps, String taskName) { + this.configProps = checkNotNull(configProps); + this.assignedShardsWithStartingSequenceNum = new HashMap<>(); + for (KinesisStreamShard shard : assignedShards) { + assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString()); + } + this.taskName = taskName; + this.error = new AtomicReference<>(); + } + + /** + * Advance a shard's starting sequence number to a specified value + * + * @param streamShard the shard to perform the advance on + * @param sequenceNum the sequence number to advance to + */ + public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) { + if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) { + throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read."); + } + assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum); + } + + public void run(SourceFunction.SourceContext sourceContext, + KinesisDeserializationSchema deserializationSchema, + HashMap lastSequenceNums) throws Exception { + + if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) { + throw new IllegalArgumentException("No shards set to read for this fetcher"); + } + + this.mainThread = Thread.currentThread(); + + LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum); + + // create a thread for each individual shard + ArrayList> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size()); + for (Map.Entry assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) { + ShardConsumerThread thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(), + assignedShard.getValue(), sourceContext, deserializationSchema, lastSequenceNums); + thread.setName(String.format("ShardConsumer - %s - %s/%s", + taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId())); + thread.setDaemon(true); + consumerThreads.add(thread); + } + + // check that we are viable for running for the last time before starting threads + if (!running) { + return; + } + + for (ShardConsumerThread shardConsumer : consumerThreads) { + LOG.info("Starting thread {}", shardConsumer.getName()); + shardConsumer.start(); + } + + // wait until all consumer threads are done, or until the fetcher is aborted, or until + // an error occurred in one of the consumer threads + try { + boolean consumersStillRunning = true; + while (running && error.get() == null && consumersStillRunning) { + try { + // wait for the consumer threads. if an error occurs, we are interrupted + for (ShardConsumerThread consumerThread : consumerThreads) { + consumerThread.join(); + } + + // check if there are consumer threads still running + consumersStillRunning = false; + for (ShardConsumerThread consumerThread : consumerThreads) { + consumersStillRunning = consumersStillRunning | consumerThread.isAlive(); + } + } catch (InterruptedException e) { + // ignore + } + } + + // make sure any asynchronous error is noticed + Throwable error = this.error.get(); + if (error != null) { + throw new Exception(error.getMessage(), error); + } + } finally { + for (ShardConsumerThread consumerThread : consumerThreads) { + if (consumerThread.isAlive()) { + consumerThread.cancel(); + } + } + } + } + + public void close() throws IOException { + this.running = false; + } + + public void stopWithError(Throwable throwable) { + if (this.error.compareAndSet(null, throwable)) { + if (mainThread != null) { + mainThread.interrupt(); + } + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java new file mode 100644 index 0000000000000..7f3144ee846b7 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.internals; + +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; + +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Properties; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Thread that does the actual data pulling from AWS Kinesis shards. Each thread is in charge of one Kinesis shard only. + */ +public class ShardConsumerThread extends Thread { + private final SourceFunction.SourceContext sourceContext; + private final KinesisDeserializationSchema deserializer; + private final HashMap seqNoState; + + private final KinesisProxy kinesisProxy; + + private final KinesisDataFetcher ownerRef; + + private final KinesisStreamShard assignedShard; + + private final int maxNumberOfRecordsPerFetch; + + private String lastSequenceNum; + private String nextShardItr; + + private volatile boolean running = true; + + public ShardConsumerThread(KinesisDataFetcher ownerRef, + Properties props, + KinesisStreamShard assignedShard, + String lastSequenceNum, + SourceFunction.SourceContext sourceContext, + KinesisDeserializationSchema deserializer, + HashMap seqNumState) { + this.ownerRef = checkNotNull(ownerRef); + this.assignedShard = checkNotNull(assignedShard); + this.lastSequenceNum = checkNotNull(lastSequenceNum); + this.sourceContext = checkNotNull(sourceContext); + this.deserializer = checkNotNull(deserializer); + this.seqNoState = checkNotNull(seqNumState); + this.kinesisProxy = new KinesisProxy(props); + this.maxNumberOfRecordsPerFetch = Integer.valueOf(props.getProperty( + KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET, + Integer.toString(KinesisConfigConstants.DEFAULT_SHARD_RECORDS_PER_GET))); + } + + @SuppressWarnings("unchecked") + @Override + public void run() { + try { + if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.toString())) { + // if the shard is already closed, there will be no latest next record to get for this shard + if (assignedShard.isClosed()) { + nextShardItr = null; + } else { + nextShardItr = kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.LATEST.toString(), null); + } + } else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.toString())) { + nextShardItr = kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.TRIM_HORIZON.toString(), null); + } else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.toString())) { + nextShardItr = null; + } else { + nextShardItr = kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum); + } + + while(running) { + if (nextShardItr == null) { + lastSequenceNum = SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.toString(); + + synchronized (sourceContext.getCheckpointLock()) { + seqNoState.put(assignedShard, lastSequenceNum); + } + + break; + } else { + GetRecordsResult getRecordsResult = kinesisProxy.getRecords(nextShardItr, maxNumberOfRecordsPerFetch); + + List fetchedRecords = getRecordsResult.getRecords(); + + // each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding + fetchedRecords = deaggregateRecords(fetchedRecords, assignedShard.getStartingHashKey(), assignedShard.getEndingHashKey()); + + for (Record record : fetchedRecords) { + ByteBuffer recordData = record.getData(); + + byte[] dataBytes = new byte[recordData.remaining()]; + recordData.get(dataBytes); + + byte[] keyBytes = record.getPartitionKey().getBytes(); + + final T value = deserializer.deserialize(keyBytes, dataBytes,assignedShard.getStreamName(), + record.getSequenceNumber()); + + synchronized (sourceContext.getCheckpointLock()) { + sourceContext.collect(value); + seqNoState.put(assignedShard, record.getSequenceNumber()); + } + + lastSequenceNum = record.getSequenceNumber(); + } + + nextShardItr = getRecordsResult.getNextShardIterator(); + } + } + } catch (Throwable t) { + ownerRef.stopWithError(t); + } + } + + public void cancel() { + this.running = false; + this.interrupt(); + } + + @SuppressWarnings("unchecked") + protected static List deaggregateRecords(List records, String startingHashKey, String endingHashKey) { + return (List) (List) UserRecord.deaggregate(records, new BigInteger(startingHashKey), new BigInteger(endingHashKey)); + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java new file mode 100644 index 0000000000000..cf0011357b04e --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.model; + +import com.amazonaws.services.kinesis.model.Shard; + +import java.io.Serializable; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A serializable representation of a AWS Kinesis Stream shard. It is basically a wrapper class around the information + * provided along with {@link com.amazonaws.services.kinesis.model.Shard}, with some extra utility methods to + * determine whether or not a shard is closed and whether or not the shard is a result of parent shard splits or merges. + */ +public class KinesisStreamShard implements Serializable { + + private static final long serialVersionUID = -6004217801761077536L; + + private final String streamName; + private final Shard shard; + + private final int cachedHash; + + /** + * Create a new KinesisStreamShard + * + * @param streamName + * the name of the Kinesis stream that this shard belongs to + * @param shard + * the actual AWS Shard instance that will be wrapped within this KinesisStreamShard + */ + public KinesisStreamShard(String streamName, Shard shard) { + this.streamName = checkNotNull(streamName); + this.shard = checkNotNull(shard); + + this.cachedHash = 37 * (streamName.hashCode() + shard.hashCode()); + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shard.getShardId(); + } + + public String getStartingSequenceNumber() { + return shard.getSequenceNumberRange().getStartingSequenceNumber(); + } + + public String getEndingSequenceNumber() { + return shard.getSequenceNumberRange().getEndingSequenceNumber(); + } + + public String getStartingHashKey() { + return shard.getHashKeyRange().getStartingHashKey(); + } + + public String getEndingHashKey() { + return shard.getHashKeyRange().getEndingHashKey(); + } + + public boolean isClosed() { + return (getEndingSequenceNumber() != null); + } + + public String getParentShardId() { + return shard.getParentShardId(); + } + + public String getAdjacentParentShardId() { + return shard.getAdjacentParentShardId(); + } + + public boolean isSplitShard() { + return (getParentShardId() != null && getAdjacentParentShardId() == null); + } + + public boolean isMergedShard() { + return (getParentShardId() != null && getAdjacentParentShardId() != null); + } + + public Shard getShard() { + return shard; + } + + @Override + public String toString() { + return "KinesisStreamShard{" + + "streamName='" + streamName + "'" + + ", shardId='" + getShardId() + "'" + + ", parentShardId='" + getParentShardId() + "'" + + ", adjacentParentShardId='" + getAdjacentParentShardId() + "'" + + ", startingSequenceNumber='" + getStartingSequenceNumber() + "'" + + ", endingSequenceNumber='" + getEndingSequenceNumber() + "'" + + ", startingHashKey='" + getStartingHashKey() + "'" + + ", endingHashKey='" + getEndingHashKey() + "'}"; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof KinesisStreamShard)) { + return false; + } + + if (obj == this) { + return true; + } + + KinesisStreamShard other = (KinesisStreamShard) obj; + + return streamName.equals(other.getStreamName()) && shard.equals(other.getShard()); + } + + @Override + public int hashCode() { + return cachedHash; + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java new file mode 100644 index 0000000000000..f18e66481c252 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.model; + +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; + +/** + * Special flag values for sequence numbers in shards to indicate special positions. + * The value is initially set by {@link FlinkKinesisConsumer} when {@link KinesisDataFetcher}s are created. + * The KinesisDataFetchers will use this value to determine how to retrieve the starting shard iterator from AWS Kinesis. + */ +public enum SentinelSequenceNumber { + + /** Flag value to indicate that the sequence number of a shard is not set. This value is used + * as an initial value in {@link KinesisDataFetcher}'s constructor for all shard's sequence number. */ + SENTINEL_SEQUENCE_NUMBER_NOT_SET, + + /** Flag value for shard's sequence numbers to indicate that the + * shard should start to be read from the latest incoming records */ + SENTINEL_LATEST_SEQUENCE_NUM, + + /** Flag value for shard's sequence numbers to indicate that the shard should + * start to be read from the earliest records that haven't expired yet */ + SENTINEL_EARLIEST_SEQUENCE_NUM, + + /** Flag value to indicate that we have already read the last record of this shard + * (Note: Kinesis shards that have been closed due to a split or merge will have an ending data record) */ + SENTINEL_SHARD_ENDING_SEQUENCE_NUM +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java new file mode 100644 index 0000000000000..3d79adc40ae08 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.proxy; + +import com.amazonaws.regions.Region; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.model.DescribeStreamRequest; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.StreamStatus; +import com.amazonaws.services.kinesis.model.Shard; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A utility class that is used as a proxy to make calls to AWS Kinesis + * for several functions, such as getting a list of shards and fetching + * a batch of data records starting from a specified record sequence number. + * + * NOTE: + * In the AWS KCL library, there is a similar implementation - {@link com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy}. + * This implementation differs mainly in that we can make operations to arbitrary Kinesis streams, which is a needed + * functionality for the Flink Kinesis Connecter since the consumer may simultaneously read from multiple Kinesis streams. + */ +public class KinesisProxy { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class); + + /** The actual Kinesis client from the AWS SDK that we will be using to make calls */ + private final AmazonKinesisClient kinesisClient; + + /** The AWS region that this proxy will be making calls to */ + private final String regionId; + + /** Configuration properties of this Flink Kinesis Connector */ + private final Properties configProps; + + /** + * Create a new KinesisProxy based on the supplied configuration properties + * + * @param configProps configuration properties containing AWS credential and AWS region info + */ + public KinesisProxy(Properties configProps) { + this.configProps = checkNotNull(configProps); + + this.regionId = configProps.getProperty(KinesisConfigConstants.CONFIG_AWS_REGION); + AmazonKinesisClient client = new AmazonKinesisClient(AWSUtil.getCredentialsProvider(configProps).getCredentials()); + client.setRegion(Region.getRegion(Regions.fromName(this.regionId))); + + this.kinesisClient = client; + } + + /** + * Get the next batch of data records using a specific shard iterator + * + * @param shardIterator a shard iterator that encodes info about which shard to read and where to start reading + * @param maxRecordsToGet the maximum amount of records to retrieve for this batch + * @return the batch of retrieved records + */ + public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) { + final GetRecordsRequest getRecordsRequest = new GetRecordsRequest(); + getRecordsRequest.setShardIterator(shardIterator); + getRecordsRequest.setLimit(maxRecordsToGet); + + GetRecordsResult getRecordsResult = null; + + int remainingRetryTimes = Integer.valueOf( + configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES))); + long describeStreamBackoffTimeInMillis = Long.valueOf( + configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF))); + + int i=0; + while (i <= remainingRetryTimes && getRecordsResult == null) { + try { + getRecordsResult = kinesisClient.getRecords(getRecordsRequest); + } catch (ProvisionedThroughputExceededException ex) { + LOG.warn("Got ProvisionedThroughputExceededException. Backing off for " + + describeStreamBackoffTimeInMillis + " millis."); + try { + Thread.sleep(describeStreamBackoffTimeInMillis); + } catch (InterruptedException interruptEx) { + // + } + } + i++; + } + + if (getRecordsResult == null) { + throw new RuntimeException("Rate Exceeded"); + } + + return getRecordsResult; + } + + /** + * Get the list of shards associated with multiple Kinesis streams + * + * @param streamNames the list of Kinesis streams + * @return a list of {@link KinesisStreamShard}s + */ + public List getShardList(List streamNames) { + List shardList = new ArrayList<>(); + + for (String stream : streamNames) { + DescribeStreamResult describeStreamResult; + String lastSeenShardId = null; + + do { + describeStreamResult = describeStream(stream, lastSeenShardId); + + List shards = describeStreamResult.getStreamDescription().getShards(); + for (Shard shard : shards) { + shardList.add(new KinesisStreamShard(stream, shard)); + } + lastSeenShardId = shards.get(shards.size() - 1).getShardId(); + } while (describeStreamResult.getStreamDescription().isHasMoreShards()); + } + return shardList; + } + + /** + * Get a shard iterator for a Kinesis shard + * + * @param shard the shard to get the iterator for + * @param shardIteratorType the iterator type to get + * @param startingSeqNum the sequence number that the iterator will start from + * @return the shard iterator + */ + public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) { + return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator(); + } + + /** + * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess. + * + * @param streamName the stream to describe + * @param startShardId which shard to start with for this describe operation (earlier shard's infos will not appear in result) + * @return the result of the describe stream operation + */ + private DescribeStreamResult describeStream(String streamName, String startShardId) { + final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest(); + describeStreamRequest.setStreamName(streamName); + describeStreamRequest.setExclusiveStartShardId(startShardId); + + DescribeStreamResult describeStreamResult = null; + String streamStatus = null; + int remainingRetryTimes = Integer.valueOf( + configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES))); + long describeStreamBackoffTimeInMillis = Long.valueOf( + configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF))); + + // Call DescribeStream, with backoff and retries (if we get LimitExceededException). + while ((remainingRetryTimes >= 0) && (describeStreamResult == null)) { + try { + describeStreamResult = kinesisClient.describeStream(describeStreamRequest); + streamStatus = describeStreamResult.getStreamDescription().getStreamStatus(); + } catch (LimitExceededException le) { + LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for " + + describeStreamBackoffTimeInMillis + " millis."); + try { + Thread.sleep(describeStreamBackoffTimeInMillis); + } catch (InterruptedException ie) { + LOG.debug("Stream " + streamName + " : Sleep was interrupted ", ie); + } + } catch (ResourceNotFoundException re) { + throw new RuntimeException("Error while getting stream details", re); + } + remainingRetryTimes--; + } + + if (streamStatus == null) { + throw new RuntimeException("Can't get stream info from after 3 retries due to LimitExceededException"); + } else if (streamStatus.equals(StreamStatus.ACTIVE.toString()) || + streamStatus.equals(StreamStatus.UPDATING.toString())) { + return describeStreamResult; + } else { + throw new RuntimeException("Stream is not Active or Updating"); + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java new file mode 100644 index 0000000000000..1fd6bf58486c4 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchema.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.serialization; + +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import java.io.IOException; +import java.io.Serializable; + +/** + * This is a deserialization schema specific for the Flink Kinesis Consumer. Different from the + * basic {@link DeserializationSchema}, this schema offers additional Kinesis-specific information + * about the record that may be useful to the user application. + * + * @param The type created by the keyed deserialization schema. + */ +public interface KinesisDeserializationSchema extends Serializable, ResultTypeQueryable { + + /** + * Deserializes a Kinesis record's bytes + * + * @param recordKey the records's key as a byte array (null if no key has been set for the record) + * @param recordValue the record's value as a byte array + * @param stream the name of the Kinesis stream that this record was sent to + * @param seqNum the sequence number of this record in the Kinesis shard + * @return the deserialized message as an Java object + * @throws IOException + */ + T deserialize(byte[] recordKey, byte[] recordValue, String stream, String seqNum) throws IOException; + + /** + * Method to decide whether the element signals the end of the stream. If + * true is returned the element won't be emitted. + * + * @param nextElement the element to test for the end-of-stream signal + * @return true if the element signals end of stream, false otherwise + */ + boolean isEndOfStream(T nextElement); +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java new file mode 100644 index 0000000000000..19c3e72390812 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisDeserializationSchemaWrapper.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.serialization; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; + +import java.io.IOException; + +/** + * A simple wrapper for using the {@link DeserializationSchema} with the {@link KinesisDeserializationSchema} interface + * + * @param The type created by the deserialization schema. + */ +public class KinesisDeserializationSchemaWrapper implements KinesisDeserializationSchema { + private static final long serialVersionUID = 9143148962928375886L; + + private final DeserializationSchema deserializationSchema; + + public KinesisDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + @Override + public T deserialize(byte[] recordKey, byte[] recordValue, String stream, String seqNu) throws IOException { + return deserializationSchema.deserialize(recordValue); + } + + @Override + public boolean isEndOfStream(T nextElement) { + return deserializationSchema.isEndOfStream(nextElement); + } + + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisSerializationSchema.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java similarity index 95% rename from flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisSerializationSchema.java rename to flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java index 86f14fe4f8f73..03dd72c13919c 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/KinesisSerializationSchema.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/KinesisSerializationSchema.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.connectors.kinesis; +package org.apache.flink.streaming.connectors.kinesis.serialization; import java.io.Serializable; diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java new file mode 100644 index 0000000000000..187f0984f47bb --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.util; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; +import com.amazonaws.auth.SystemPropertiesCredentialsProvider; +import com.amazonaws.auth.profile.ProfileCredentialsProvider; +import com.amazonaws.regions.Regions; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.CredentialProviderType; + +import java.util.Properties; + +/** + * Some utilities specific to Amazon Web Service. + */ +public class AWSUtil { + + /** + * Return a {@link AWSCredentialsProvider} instance corresponding to the configuration properties. + * + * @param configProps the configuration properties + * @return The corresponding AWS Credentials Provider instance + */ + public static AWSCredentialsProvider getCredentialsProvider(final Properties configProps) { + CredentialProviderType credentialProviderType = CredentialProviderType.valueOf(configProps.getProperty( + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE, CredentialProviderType.BASIC.toString())); + + AWSCredentialsProvider credentialsProvider; + + switch (credentialProviderType) { + case ENV_VAR: + credentialsProvider = new EnvironmentVariableCredentialsProvider(); + break; + case SYS_PROP: + credentialsProvider = new SystemPropertiesCredentialsProvider(); + break; + case PROFILE: + String profileName = configProps.getProperty( + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_PROFILE_NAME, null); + String profileConfigPath = configProps.getProperty( + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_PROFILE_PATH, null); + credentialsProvider = (profileConfigPath == null) + ? new ProfileCredentialsProvider(profileName) + : new ProfileCredentialsProvider(profileConfigPath, profileName); + break; + default: + case BASIC: + credentialsProvider = new AWSCredentialsProvider() { + @Override + public AWSCredentials getCredentials() { + return new BasicAWSCredentials( + configProps.getProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID), + configProps.getProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY)); + } + + @Override + public void refresh() { + // do nothing + } + }; + } + + return credentialsProvider; + } + + /** + * Checks whether or not a region ID is valid + * + * @param region The AWS region ID to check + * @return true if the supplied region ID is valid, false otherwise + */ + public static boolean isValidRegion(String region) { + try { + Regions.fromName(region.toLowerCase()); + } catch (IllegalArgumentException e) { + return false; + } + return true; + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java new file mode 100644 index 0000000000000..d68182c4225fa --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java @@ -0,0 +1,507 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.testutils.ReferenceKinesisShardTopologies; +import org.apache.flink.streaming.connectors.kinesis.testutils.TestableFlinkKinesisConsumer; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.lang.reflect.Field; +import java.util.Properties; +import java.util.List; +import java.util.ArrayList; +import java.util.UUID; +import java.util.HashMap; +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.whenNew; + +/** + * Suite of FlinkKinesisConsumer tests, including utility static method tests, + * and tests for the methods called throughout the source life cycle with mocked KinesisProxy. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(FlinkKinesisConsumer.class) +public class FlinkKinesisConsumerTest { + + @Rule + private ExpectedException exception = ExpectedException.none(); + + // ---------------------------------------------------------------------- + // FlinkKinesisConsumer.validatePropertiesConfig() tests + // ---------------------------------------------------------------------- + + @Test + public void testMissingAwsRegionInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("AWS region must be set"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testUnrecognizableAwsRegionInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid AWS region"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "wrongRegionId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testCredentialProviderTypeDefaultToBasicButNoCredentialsSetInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Need to set values for AWS Access Key ID and Secret Key"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testCredentialProviderTypeSetToBasicButNoCredentialSetInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Need to set values for AWS Access Key ID and Secret Key"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE, "BASIC"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testUnrecognizableCredentialProviderTypeInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid AWS Credential Provider Type"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE, "wrongProviderType"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testUnrecognizableStreamInitPositionTypeInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid initial position in stream"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE, "BASIC"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "wrongInitPosition"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testUnparsableIntForDescribeStreamRetryCountInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for describeStream stream operation retry count"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, "unparsableInt"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testUnparsableLongForDescribeStreamBackoffMillisInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for describeStream stream operation backoff milliseconds"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, "unparsableLong"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + @Test + public void testUnparsableIntForGetRecordsMaxCountInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for maximum records per getRecords shard operation"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET, "unparsableInt"); + + FlinkKinesisConsumer.validatePropertiesConfig(testConfig); + } + + // ---------------------------------------------------------------------- + // FlinkKinesisConsumer.assignShards() tests + // ---------------------------------------------------------------------- + + @Test + public void testShardNumEqualConsumerNum() { + try { + List fakeShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + int consumerTaskCount = fakeShards.size(); + + for (int consumerNum=0; consumerNum < consumerTaskCount; consumerNum++) { + List assignedShardsToThisConsumerTask = + FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); + + // the ith consumer should be assigned exactly 1 shard, + // which is always the ith shard of a shard list that only has open shards + assertEquals(1, assignedShardsToThisConsumerTask.size()); + assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(consumerNum))); + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testShardNumFewerThanConsumerNum() { + try { + List fakeShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + int consumerTaskCount = fakeShards.size() + 3; + + for (int consumerNum = 0; consumerNum < consumerTaskCount; consumerNum++) { + List assignedShardsToThisConsumerTask = + FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); + + // for ith consumer with i < the total num of shards, + // the ith consumer should be assigned exactly 1 shard, + // which is always the ith shard of a shard list that only has open shards; + // otherwise, the consumer should not be assigned any shards + if (consumerNum < fakeShards.size()) { + assertEquals(1, assignedShardsToThisConsumerTask.size()); + assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(consumerNum))); + } else { + assertEquals(0, assignedShardsToThisConsumerTask.size()); + } + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testShardNumMoreThanConsumerNum() { + try { + List fakeShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + int consumerTaskCount = fakeShards.size() - 1; + + for (int consumerNum = 0; consumerNum < consumerTaskCount; consumerNum++) { + List assignedShardsToThisConsumerTask = + FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); + + // since the number of consumer tasks is short by 1, + // all but the first consumer task should be assigned 1 shard, + // while the first consumer task is assigned 2 shards + if (consumerNum != 0) { + assertEquals(1, assignedShardsToThisConsumerTask.size()); + assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(consumerNum))); + } else { + assertEquals(2, assignedShardsToThisConsumerTask.size()); + assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(0))); + assertTrue(assignedShardsToThisConsumerTask.get(1).equals(fakeShards.get(fakeShards.size()-1))); + } + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + @Test + public void testAssignEmptyShards() { + try { + List fakeShards = new ArrayList<>(0); + int consumerTaskCount = 4; + + for (int consumerNum = 0; consumerNum < consumerTaskCount; consumerNum++) { + List assignedShardsToThisConsumerTask = + FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); + + // should not be assigned anything + assertEquals(0, assignedShardsToThisConsumerTask.size()); + + } + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + // ---------------------------------------------------------------------- + // Constructor tests with mocked KinesisProxy + // ---------------------------------------------------------------------- + + @Test + public void testConstructorShouldThrowRuntimeExceptionIfUnableToFindAnyShards() { + exception.expect(RuntimeException.class); + exception.expectMessage("Unable to retrieve any shards"); + + Properties testConsumerConfig = new Properties(); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + + // get a consumer that will not be able to find any shards from AWS Kinesis + FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( + 6, 2, "fake-consumer-task-name", + new ArrayList(), new ArrayList(), testConsumerConfig, + null, null, false, false); + } + + // ---------------------------------------------------------------------- + // Tests for open() source life cycle method + // ---------------------------------------------------------------------- + + @Test + public void testOpenWithNoRestoreStateFetcherAdvanceToLatestSentinelSequenceNumberWhenConfigSetToStartFromLatest() throws Exception { + + int fakeNumConsumerTasks = 6; + int fakeThisConsumerTaskIndex = 2; + String fakeThisConsumerTaskName = "fake-this-task-name"; + + List fakeCompleteShardList = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + List fakeAssignedShardsToThisConsumerTask = fakeCompleteShardList.subList(2,3); + + Properties testConsumerConfig = new Properties(); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "LATEST"); + + KinesisDataFetcher kinesisDataFetcherMock = Mockito.mock(KinesisDataFetcher.class); + try { + whenNew(KinesisDataFetcher.class).withArguments(fakeAssignedShardsToThisConsumerTask, testConsumerConfig, fakeThisConsumerTaskName).thenReturn(kinesisDataFetcherMock); + } catch (Exception e) { + throw new RuntimeException("Error when power mocking KinesisDataFetcher in test", e); + } + + FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( + fakeNumConsumerTasks, fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, + fakeCompleteShardList, fakeAssignedShardsToThisConsumerTask, testConsumerConfig, + null, null, false, false); + + dummyConsumer.open(new Configuration()); + + for (KinesisStreamShard shard : fakeAssignedShardsToThisConsumerTask) { + verify(kinesisDataFetcherMock).advanceSequenceNumberTo(shard, SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.toString()); + } + + } + + @Test + public void testOpenWithNoRestoreStateFetcherAdvanceToEarliestSentinelSequenceNumberWhenConfigSetToTrimHorizon() throws Exception { + + int fakeNumConsumerTasks = 6; + int fakeThisConsumerTaskIndex = 2; + String fakeThisConsumerTaskName = "fake-this-task-name"; + + List fakeCompleteShardList = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + List fakeAssignedShardsToThisConsumerTask = fakeCompleteShardList.subList(2,3); + + Properties testConsumerConfig = new Properties(); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "TRIM_HORIZON"); + + KinesisDataFetcher kinesisDataFetcherMock = Mockito.mock(KinesisDataFetcher.class); + try { + whenNew(KinesisDataFetcher.class).withArguments(fakeAssignedShardsToThisConsumerTask, testConsumerConfig, fakeThisConsumerTaskName).thenReturn(kinesisDataFetcherMock); + } catch (Exception e) { + throw new RuntimeException("Error when power mocking KinesisDataFetcher in test", e); + } + + FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( + fakeNumConsumerTasks, fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, + fakeCompleteShardList, fakeAssignedShardsToThisConsumerTask, testConsumerConfig, + null, null, false, false); + + dummyConsumer.open(new Configuration()); + + for (KinesisStreamShard shard : fakeAssignedShardsToThisConsumerTask) { + verify(kinesisDataFetcherMock).advanceSequenceNumberTo(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.toString()); + } + + } + + @Test + public void testOpenWithRestoreStateFetcherAdvanceToCorrespondingSequenceNumbers() throws Exception { + + int fakeNumConsumerTasks = 6; + int fakeThisConsumerTaskIndex = 2; + String fakeThisConsumerTaskName = "fake-this-task-name"; + + List fakeCompleteShardList = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + List fakeAssignedShardsToThisConsumerTask = fakeCompleteShardList.subList(2,3); + + Properties testConsumerConfig = new Properties(); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "TRIM_HORIZON"); + + KinesisDataFetcher kinesisDataFetcherMock = Mockito.mock(KinesisDataFetcher.class); + try { + whenNew(KinesisDataFetcher.class).withArguments(fakeAssignedShardsToThisConsumerTask, testConsumerConfig, fakeThisConsumerTaskName).thenReturn(kinesisDataFetcherMock); + } catch (Exception e) { + throw new RuntimeException("Error when power mocking KinesisDataFetcher in test", e); + } + + FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( + fakeNumConsumerTasks, fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, + fakeCompleteShardList, fakeAssignedShardsToThisConsumerTask, testConsumerConfig, + null, null, false, false); + + // generate random UUIDs as sequence numbers of last checkpointed state for each assigned shard + ArrayList listOfSeqNumIfAssignedShards = new ArrayList<>(fakeAssignedShardsToThisConsumerTask.size()); + for (KinesisStreamShard shard : fakeAssignedShardsToThisConsumerTask) { + listOfSeqNumIfAssignedShards.add(UUID.randomUUID().toString()); + } + + HashMap fakeRestoredState = new HashMap<>(); + for (int i=0; i fakeCompleteShardList, + List fakeAssignedShardListToThisConsumerTask, + Properties consumerTestConfig, + KinesisDataFetcher fetcher, + HashMap lastSequenceNumsToRestore, + boolean hasAssignedShards, + boolean running) { + + final String dummyKinesisStreamName = "flink-test"; + + final List dummyKinesisStreamList = Collections.singletonList(dummyKinesisStreamName); + + final KinesisProxy kinesisProxyMock = mock(KinesisProxy.class); + + // mock KinesisProxy that is instantiated in the constructor, as well as its getShardList call + try { + whenNew(KinesisProxy.class).withArguments(consumerTestConfig).thenReturn(kinesisProxyMock); + } catch (Exception e) { + throw new RuntimeException("Error when power mocking KinesisProxy in tests", e); + } + + when(kinesisProxyMock.getShardList(dummyKinesisStreamList)).thenReturn(fakeCompleteShardList); + + TestableFlinkKinesisConsumer dummyConsumer = + new TestableFlinkKinesisConsumer(dummyKinesisStreamName, fakeNumFlinkConsumerTasks, + fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, consumerTestConfig); + + try { + Field fetcherField = FlinkKinesisConsumer.class.getDeclaredField("fetcher"); + fetcherField.setAccessible(true); + fetcherField.set(dummyConsumer, fetcher); + + Field lastSequenceNumsField = FlinkKinesisConsumer.class.getDeclaredField("lastSequenceNums"); + lastSequenceNumsField.setAccessible(true); + lastSequenceNumsField.set(dummyConsumer, lastSequenceNumsToRestore); + + Field hasAssignedShardsField = FlinkKinesisConsumer.class.getDeclaredField("hasAssignedShards"); + hasAssignedShardsField.setAccessible(true); + hasAssignedShardsField.set(dummyConsumer, hasAssignedShards); + + Field runningField = FlinkKinesisConsumer.class.getDeclaredField("running"); + runningField.setAccessible(true); + runningField.set(dummyConsumer, running); + } catch (IllegalAccessException | NoSuchFieldException e) { + // no reason to end up here ... + throw new RuntimeException(e); + } + + // mock FlinkKinesisConsumer utility static methods + mockStatic(FlinkKinesisConsumer.class); + + try { + // assume assignShards static method is correct by mocking + PowerMockito.when( + FlinkKinesisConsumer.assignShards( + fakeCompleteShardList, + fakeNumFlinkConsumerTasks, + fakeThisConsumerTaskIndex)) + .thenReturn(fakeAssignedShardListToThisConsumerTask); + + // assume validatePropertiesConfig static method is correct by mocking + PowerMockito.doNothing().when(FlinkKinesisConsumer.class, "validatePropertiesConfig", Mockito.any(Properties.class)); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException("Error when power mocking static methods of FlinkKinesisConsumer", e); + } + + return dummyConsumer; + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java new file mode 100644 index 0000000000000..74734033393c1 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.internals; + +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.testutils.ReferenceKinesisShardTopologies; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.List; +import java.util.Properties; + +public class KinesisDataFetcherTest { + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void testAdvanceSequenceNumberOnNotOwnedShard() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Can't advance sequence number on a shard we are not going to read."); + + List fakeCompleteListOfShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); + List assignedShardsToThisFetcher = fakeCompleteListOfShards.subList(0,1); + + KinesisDataFetcher fetcherUnderTest = new KinesisDataFetcher(assignedShardsToThisFetcher, new Properties(), "fake-task-name"); + + // advance the fetcher on a shard that it does not own + fetcherUnderTest.advanceSequenceNumberTo(fakeCompleteListOfShards.get(2), "fake-seq-num"); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java new file mode 100644 index 0000000000000..93467a07d31ff --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.internals; + +import com.amazonaws.services.kinesis.model.*; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Matchers; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Properties; +import java.util.HashMap; +import java.util.UUID; +import java.util.LinkedList; + +import static org.junit.Assert.assertEquals; + +/** + * Tests on how the ShardConsumerThread behaves with mocked KinesisProxy behaviours. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(ShardConsumerThread.class) +public class ShardConsumerThreadTest { + + @Test + public void testAllRecordsFetchedFromKinesisAreCorrectlyCollected() { + int totalRecordCount = 500; + + KinesisStreamShard assignedShardUnderTest = new KinesisStreamShard( + "fake-stream-name", + new Shard() + .withShardId("fake-shard-id") + .withAdjacentParentShardId(null) + .withParentShardId(null) + .withHashKeyRange(new HashKeyRange().withStartingHashKey("0").withEndingHashKey(StringUtils.repeat("FF", 16)))); + + // ------------------------------------------------------------------------------------------ + // the part below prepares the behaviour of the mocked KinesisProxy for getting the inital shard iterator, + // followed by consecutive getRecords() calls until total of 500 records fetched + // ------------------------------------------------------------------------------------------ + + KinesisProxy kinesisProxyMock = Mockito.mock(KinesisProxy.class); + Mockito.when(kinesisProxyMock.getShardIterator(Matchers.any(KinesisStreamShard.class), Matchers.anyString(), Matchers.anyString())) + .thenReturn("fake-initial-shard-itr"); + + // 1st getRecords() returns 100 records + GetRecordsResult getRecordsResultFirst = new GetRecordsResult() + .withRecords(generateFakeListOfRecordsFromToIncluding(0, 99)) + .withNextShardIterator("fake-1st-shard-itr"); + + // 2nd getRecords() returns 90 records + GetRecordsResult getRecordsResultSecond = new GetRecordsResult() + .withRecords(generateFakeListOfRecordsFromToIncluding(100, 189)) + .withNextShardIterator("fake-2nd-shard-itr"); + + // 3rd getRecords() returns 78 records + GetRecordsResult getRecordsResultThird = new GetRecordsResult() + .withRecords(generateFakeListOfRecordsFromToIncluding(190, 267)) + .withNextShardIterator("fake-3rd-shard-itr"); + + // 4th getRecords() returns 100 records + GetRecordsResult getRecordsResultFourth = new GetRecordsResult() + .withRecords(generateFakeListOfRecordsFromToIncluding(268, 367)) + .withNextShardIterator("fake-4th-shard-itr"); + + GetRecordsResult getRecordsResultFifth = new GetRecordsResult() + .withRecords(generateFakeListOfRecordsFromToIncluding(368, 459)) + .withNextShardIterator("fake-5th-shard-itr"); + + GetRecordsResult getRecordsResultFinal = new GetRecordsResult() + .withRecords(generateFakeListOfRecordsFromToIncluding(460, 499)) + .withNextShardIterator(null); + + Mockito.when(kinesisProxyMock.getRecords(Matchers.anyString(), Matchers.anyInt())) + .thenReturn(getRecordsResultFirst) + .thenReturn(getRecordsResultSecond) + .thenReturn(getRecordsResultThird) + .thenReturn(getRecordsResultFourth) + .thenReturn(getRecordsResultFifth) + .thenReturn(getRecordsResultFinal); + + // assuming that all fetched records are not aggregated, + // so we are mocking the static deaggregateRecords() to return the original list of records + PowerMockito.mockStatic(ShardConsumerThread.class); + PowerMockito.when(ShardConsumerThread.deaggregateRecords(Matchers.anyListOf(Record.class), Matchers.anyString(), Matchers.anyString())) + .thenReturn(getRecordsResultFirst.getRecords()) + .thenReturn(getRecordsResultSecond.getRecords()) + .thenReturn(getRecordsResultThird.getRecords()) + .thenReturn(getRecordsResultFourth.getRecords()) + .thenReturn(getRecordsResultFifth.getRecords()) + .thenReturn(getRecordsResultFinal.getRecords()); + + // ------------------------------------------------------------------------------------------ + + Properties testConsumerConfig = new Properties(); + HashMap seqNumState = new HashMap<>(); + + DummySourceContext dummySourceContext = new DummySourceContext(); + ShardConsumerThread dummyShardConsumerThread = getDummyShardConsumerThreadWithMockedKinesisProxy( + dummySourceContext, kinesisProxyMock, Mockito.mock(KinesisDataFetcher.class), + testConsumerConfig, assignedShardUnderTest, "fake-last-seq-num", seqNumState); + + dummyShardConsumerThread.run(); + + // the final sequence number state for the assigned shard to this consumer thread + // should store SENTINEL_SHARD_ENDING_SEQUENCE_NUMBER since the final nextShardItr should be null + assertEquals(seqNumState.get(assignedShardUnderTest), SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.toString()); + + // the number of elements collected should equal the number of records generated by mocked KinesisProxy + assertEquals(dummySourceContext.getNumOfElementsCollected(), totalRecordCount); + } + + private ShardConsumerThread getDummyShardConsumerThreadWithMockedKinesisProxy( + SourceFunction.SourceContext dummySourceContext, + KinesisProxy kinesisProxyMock, + KinesisDataFetcher owningFetcherRefMock, + Properties testConsumerConfig, + KinesisStreamShard assignedShard, + String lastSequenceNum, + HashMap seqNumState) { + + try { + PowerMockito.whenNew(KinesisProxy.class).withArguments(testConsumerConfig).thenReturn(kinesisProxyMock); + } catch (Exception e) { + throw new RuntimeException("Error when power mocking KinesisProxy in test", e); + } + + return new ShardConsumerThread<>(owningFetcherRefMock, testConsumerConfig, + assignedShard, lastSequenceNum, dummySourceContext, new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()), seqNumState); + } + + private List generateFakeListOfRecordsFromToIncluding(int startingSeq, int endingSeq) { + List fakeListOfRecords = new LinkedList<>(); + for (int i=0; i <= (endingSeq - startingSeq); i++) { + fakeListOfRecords.add(new Record() + .withData(ByteBuffer.wrap(String.valueOf(i).getBytes())) + .withPartitionKey(UUID.randomUUID().toString()) // the partition key assigned doesn't matter here + .withSequenceNumber(String.valueOf(i))); // assign the order of the record within the whole sequence as the sequence num + } + return fakeListOfRecords; + } + + private static class DummySourceContext implements SourceFunction.SourceContext { + private static final Object lock = new Object(); + + private static long numElementsCollected; + + public DummySourceContext() { + numElementsCollected = 0; + } + + @Override + public void collect(String element) { + numElementsCollected++; + } + + @Override + public void collectWithTimestamp(java.lang.String element, long timestamp) { + } + + @Override + public void emitWatermark(Watermark mark) { + } + + @Override + public Object getCheckpointLock() { + return lock; + } + + @Override + public void close() { + } + + public long getNumOfElementsCollected() { + return numElementsCollected; + } + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java new file mode 100644 index 0000000000000..68ac4e5d2b6dd --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kinesis.manualtests; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; +import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.util.Collector; + +import java.nio.ByteBuffer; +import java.util.Properties; + +/** + * This is a manual test for the AWS Kinesis connector in Flink. + * + * It uses: + * - A custom KinesisSerializationSchema + * - A custom KinesisPartitioner + * + * Invocation: + * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX + */ +public class ManualConsumerProducerTest { + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.setParallelism(4); + + DataStream simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator()); + + FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(pt.getRequired("region"), + pt.getRequired("accessKey"), + pt.getRequired("secretKey"), + new KinesisSerializationSchema() { + @Override + public ByteBuffer serialize(String element) { + return ByteBuffer.wrap(element.getBytes()); + } + + // every 10th element goes into a different stream + @Override + public String getTargetStream(String element) { + if(element.split("-")[0].endsWith("0")) { + return "flink-test-2"; + } + return null; // send to default stream + } + }); + + kinesis.setFailOnError(true); + kinesis.setDefaultStream("test-flink"); + kinesis.setDefaultPartition("0"); + kinesis.setCustomPartitioner(new KinesisPartitioner() { + @Override + public String getPartitionId(String element) { + int l = element.length(); + return element.substring(l - 1, l); + } + }); + simpleStringStream.addSink(kinesis); + + + // consuming topology + Properties consumerProps = new Properties(); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accessKey")); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretKey")); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, pt.getRequired("region")); + DataStream consuming = see.addSource(new FlinkKinesisConsumer<>("test-flink", new SimpleStringSchema(), consumerProps)); + // validate consumed records for correctness + consuming.flatMap(new FlatMapFunction() { + @Override + public void flatMap(String value, Collector out) throws Exception { + String[] parts = value.split("-"); + try { + long l = Long.parseLong(parts[0]); + if(l < 0) { + throw new RuntimeException("Negative"); + } + } catch(NumberFormatException nfe) { + throw new RuntimeException("First part of '" + value + "' is not a valid numeric type"); + } + if(parts[1].length() != 12) { + throw new RuntimeException("Second part of '" + value + "' doesn't have 12 characters"); + } + } + }); + consuming.print(); + + see.execute(); + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java new file mode 100644 index 0000000000000..d8c277069554d --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.streaming.connectors.kinesis.manualtests; + +import com.amazonaws.regions.Region; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; +import org.apache.flink.streaming.connectors.kinesis.config.InitialPosition; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.BitSet; +import java.util.Properties; +import java.util.UUID; + +import static org.apache.flink.test.util.TestUtils.tryExecute; + +/** + * This test first starts a data generator, producing data into kinesis. + * Then, it starts a consuming topology, ensuring that all records up to a certain + * point have been seen. + * + * Invocation: + * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX + */ +public class ManualExactlyOnceTest { + + private static final Logger LOG = LoggerFactory.getLogger(ManualExactlyOnceTest.class); + + static final long TOTAL_EVENT_COUNT = 1000; // the producer writes one per 10 ms, so it runs for 10k ms = 10 seconds + + public static void main(String[] args) throws Exception { + final ParameterTool pt = ParameterTool.fromArgs(args); + LOG.info("Starting exactly once test"); + + // create a stream for the test: + Properties configProps = new Properties(); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accessKey")); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretKey")); + AmazonKinesisClient client = new AmazonKinesisClient(AWSUtil.getCredentialsProvider(configProps).getCredentials()); + client.setRegion(Region.getRegion(Regions.fromName(pt.getRequired("region")))); + final String streamName = "flink-test-" + UUID.randomUUID().toString(); + client.createStream(streamName, 1); + // wait until stream has been created + DescribeStreamResult status = client.describeStream(streamName); + LOG.info("status {}" ,status); + while(!status.getStreamDescription().getStreamStatus().equals("ACTIVE")) { + status = client.describeStream(streamName); + LOG.info("Status of stream {}", status); + Thread.sleep(1000); + } + + final Configuration flinkConfig = new Configuration(); + flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); + flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); + flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); + + ForkableFlinkMiniCluster flink = new ForkableFlinkMiniCluster(flinkConfig, false); + flink.start(); + + final int flinkPort = flink.getLeaderRPCPort(); + + try { + final Tuple1 producerException = new Tuple1<>(); + Runnable producer = new Runnable() { + @Override + public void run() { + try { + StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); + see.setParallelism(2); + + // start data generator + DataStream simpleStringStream = see.addSource(new EventsGenerator(TOTAL_EVENT_COUNT)).setParallelism(1); + + FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(pt.getRequired("region"), + pt.getRequired("accessKey"), + pt.getRequired("secretKey"), + new SimpleStringSchema()); + + kinesis.setFailOnError(true); + kinesis.setDefaultStream(streamName); + kinesis.setDefaultPartition("0"); + simpleStringStream.addSink(kinesis); + + LOG.info("Starting producing topology"); + see.execute("Producing topology"); + LOG.info("Producing topo finished"); + } catch (Exception e) { + LOG.warn("Error while running producing topology", e); + producerException.f0 = e; + } + } + }; + Thread producerThread = new Thread(producer); + producerThread.start(); + + + final Tuple1 consumerException = new Tuple1<>(); + Runnable consumer = new Runnable() { + @Override + public void run() { + try { + StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); + see.setParallelism(2); + see.enableCheckpointing(500); + // we restart two times + see.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 500L)); + + // consuming topology + Properties consumerProps = new Properties(); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accessKey")); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretKey")); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, pt.getRequired("region")); + // start reading from beginning + consumerProps.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, InitialPosition.TRIM_HORIZON.name()); + DataStream consuming = see.addSource(new FlinkKinesisConsumer<>(streamName, new SimpleStringSchema(), consumerProps)); + consuming.flatMap(new RichFlatMapFunction() { + int count = 0; + + @Override + public void flatMap(String value, Collector out) throws Exception { + if (count++ >= 200 && getRuntimeContext().getAttemptNumber() == 0) { + throw new RuntimeException("Artificial failure. Restart pls"); + } + out.collect(value); + } + }).flatMap(new ExactlyOnceValidatingMapper()); + // validate consumed records for correctness + LOG.info("Starting consuming topology"); + tryExecute(see, "Consuming topo"); + LOG.info("Consuming topo finished"); + } catch (Exception e) { + LOG.warn("Error while running consuming topology", e); + consumerException.f0 = e; + } + } + }; + + Thread consumerThread = new Thread(consumer); + consumerThread.start(); + + long deadline = System.currentTimeMillis() + (1000 * 2 * 60); // wait at most for two minutes + while (consumerThread.isAlive() || producerThread.isAlive()) { + Thread.sleep(1000); + if (System.currentTimeMillis() >= deadline) { + LOG.warn("Deadline passed"); + break; // enough waiting + } + } + + if (producerThread.isAlive()) { + producerThread.interrupt(); + } + + if (consumerThread.isAlive()) { + consumerThread.interrupt(); + } + + if (producerException.f0 != null) { + throw new RuntimeException("Producer failed", producerException.f0); + } + if (consumerException.f0 != null) { + throw new RuntimeException("Consumer failed", consumerException.f0); + } + + + + LOG.info("+++ TEST passed! +++"); + + } finally { + client.deleteStream(streamName); + client.shutdown(); + + // stopping flink + flink.stop(); + } + } + + // validate exactly once + private static class ExactlyOnceValidatingMapper implements FlatMapFunction, Checkpointed { + BitSet validator = new BitSet((int)TOTAL_EVENT_COUNT); + @Override + public void flatMap(String value, Collector out) throws Exception { + LOG.info("Consumed {}", value); + + int id = Integer.parseInt(value.split("-")[0]); + if(validator.get(id)) { + throw new RuntimeException("Saw id " + id +" twice!"); + } + validator.set(id); + if(id > 999) { + throw new RuntimeException("Out of bounds ID observed"); + } + + if(validator.nextClearBit(0) == 1000) { // 0 - 1000 are set + throw new SuccessException(); + } + } + + @Override + public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return validator; + } + + @Override + public void restoreState(BitSet state) throws Exception { + this.validator = state; + } + } + + public static class EventsGenerator implements SourceFunction { + private boolean running = true; + private final long limit; + + public EventsGenerator(long limit) { + this.limit = limit; + } + + + @Override + public void run(SourceContext ctx) throws Exception { + long seq = 0; + while(running) { + Thread.sleep(10); + String evt = (seq++) + "-" + RandomStringUtils.randomAlphabetic(12); + ctx.collect(evt); + LOG.info("Emitting event {}", evt); + if(seq >= limit) { + break; + } + } + ctx.close(); + LOG.info("Stopping events generator"); + } + + @Override + public void cancel() { + running = false; + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/ManualTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java similarity index 85% rename from flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/ManualTest.java rename to flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java index d846d9d5aa6d8..08f267dfe829d 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/ManualTest.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java @@ -14,12 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.streaming.connectors.kinesis; +package org.apache.flink.streaming.connectors.kinesis.manualtests; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; +import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner; import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; import java.nio.ByteBuffer; @@ -30,10 +33,12 @@ * - A custom KinesisSerializationSchema * - A custom KinesisPartitioner * + * The streams "test-flink" and "flink-test-2" must exist. + * * Invocation: * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX */ -public class ManualTest { +public class ManualProducerTest { public static void main(String[] args) throws Exception { ParameterTool pt = ParameterTool.fromArgs(args); @@ -55,7 +60,7 @@ public ByteBuffer serialize(String element) { // every 10th element goes into a different stream @Override public String getTargetStream(String element) { - if(element.endsWith("0")) { + if(element.split("-")[0].endsWith("0")) { return "flink-test-2"; } return null; // send to default stream diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java new file mode 100644 index 0000000000000..1160a6de66b67 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.testutils; + +public class KinesisShardIdGenerator { + public static String generateFromShardOrder(int order) { + return String.format("shard-%05d", order); + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java new file mode 100644 index 0000000000000..c6e8a412d4ec5 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.testutils; + +import com.amazonaws.services.kinesis.model.SequenceNumberRange; +import com.amazonaws.services.kinesis.model.Shard; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; + +import java.util.ArrayList; +import java.util.List; + +/** + * Util class to help generate example shard topologies for testing. + */ +public class ReferenceKinesisShardTopologies { + + private static final String DEFAULT_STREAM = "flink-kinesis-test"; + + /** + * A basic topology with 4 shards, where each shard is still open, + * and have no parent-child relationships due to shard split or merge. + * + * Topology layout: + * + * +- shard 0 (seq: 0 ~ open) + * | + * +- shard 1 (seq: 250 ~ open) + * | + * +- shard 2 (seq: 500 ~ open) + * | + * +- shard 3 (seq: 750 ~ open) + * + */ + public static List flatTopologyWithFourOpenShards() { + int shardCount = 4; + List topology = new ArrayList<>(shardCount); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("0")))); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(1)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("250")))); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(2)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("500")))); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(3)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("750")))); + return topology; + } + + /** + * A basic topology with 4 shards, where each shard is still open, + * and have no parent-child relationships due to shard split or merge. + * + * Topology layout: + * + * +- shard 0 (seq: 0 ~ 120) --+ + * | +- (merge) -- shard 3 (750 ~ open) + * +- shard 1 (seq: 250 ~ 289) --+ + * | + * +- shard 2 (seq: 500 ~ open) + * + */ + public static List topologyWithThreeInitialShardsAndFirstTwoMerged() { + int shardCount = 4; + + List topology = new ArrayList<>(shardCount); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("0").withEndingSequenceNumber("120")))); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(1)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("250").withEndingSequenceNumber("289")))); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(2)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("500")))); + topology.add(new KinesisStreamShard( + DEFAULT_STREAM, + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(3)) + .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("750")))); + + return topology; + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java new file mode 100644 index 0000000000000..f63ec987dd084 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kinesis.testutils; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.mockito.Mockito; + +import java.util.Properties; + +/** + * A testable FlinkKinesisConsumer that overrides getRuntimeContext to return a dummy StreamRuntimeContext. + */ +public class TestableFlinkKinesisConsumer extends FlinkKinesisConsumer { + + private final int fakeNumFlinkConsumerTasks; + private final int fakeThisConsumerTaskIndex; + private final String fakeThisConsumerTaskName; + + + public TestableFlinkKinesisConsumer(String fakeStreamName, + int fakeNumFlinkConsumerTasks, + int fakeThisConsumerTaskIndex, + String fakeThisConsumerTaskName, + Properties configProps) { + super(fakeStreamName, new SimpleStringSchema(), configProps); + this.fakeNumFlinkConsumerTasks = fakeNumFlinkConsumerTasks; + this.fakeThisConsumerTaskIndex = fakeThisConsumerTaskIndex; + this.fakeThisConsumerTaskName = fakeThisConsumerTaskName; + } + + @Override + public RuntimeContext getRuntimeContext() { + StreamingRuntimeContext runtimeContextMock = Mockito.mock(StreamingRuntimeContext.class); + Mockito.when(runtimeContextMock.getNumberOfParallelSubtasks()).thenReturn(fakeNumFlinkConsumerTasks); + Mockito.when(runtimeContextMock.getIndexOfThisSubtask()).thenReturn(fakeThisConsumerTaskIndex); + Mockito.when(runtimeContextMock.getTaskName()).thenReturn(fakeThisConsumerTaskName); + return runtimeContextMock; + } + +} From b8586f37e09da92f5b935892a368de65edcb01ec Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 29 Apr 2016 15:20:18 +0200 Subject: [PATCH 30/70] [FLINK-3229] Shade Google Protobuf into Kinesis connector to avoid runtime version conflict --- .../flink-connector-kinesis/pom.xml | 34 +++++++++++++++++++ .../kinesis/FlinkKinesisConsumer.java | 8 +++-- .../kinesis/FlinkKinesisProducer.java | 2 ++ .../kinesis/internals/KinesisDataFetcher.java | 3 +- 4 files changed, 43 insertions(+), 4 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kinesis/pom.xml b/flink-streaming-connectors/flink-connector-kinesis/pom.xml index ce49fd7d95ab5..60cbe2e218ee8 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/pom.xml +++ b/flink-streaming-connectors/flink-connector-kinesis/pom.xml @@ -121,6 +121,40 @@ under the License. + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + true + + + com.amazonaws:* + com.google.protobuf:* + + + + + + org.objectweb.asm + org.apache.flink.shaded.org.objectweb.asm + + + com.google.protobuf + org.apache.flink.kinesis.shaded.com.google.protobuf + + + + + + diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java index 97cd048583407..10162513518c6 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java @@ -412,7 +412,8 @@ protected static void validatePropertiesConfig(Properties config) { // so the Access Key ID and Secret Key must be given if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID) || !config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY)) { - throw new IllegalArgumentException("Need to set values for AWS Access Key ID and Secret Key when using the BASIC AWS credential provider type."); + throw new IllegalArgumentException("Please set values for AWS Access Key ID ('"+KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID+"') " + + "and Secret Key ('" + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY + "') when using the BASIC AWS credential provider type."); } } else { String credentialsProviderType = config.getProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE); @@ -433,13 +434,14 @@ protected static void validatePropertiesConfig(Properties config) { if (providerType == CredentialProviderType.BASIC) { if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID) || !config.containsKey(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY)) { - throw new IllegalArgumentException("Need to set values for AWS Access Key ID and Secret Key when using the BASIC AWS credential provider type."); + throw new IllegalArgumentException("Please set values for AWS Access Key ID ('"+KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID+"') " + + "and Secret Key ('" + KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY + "') when using the BASIC AWS credential provider type."); } } } if (!config.containsKey(KinesisConfigConstants.CONFIG_AWS_REGION)) { - throw new IllegalArgumentException("The AWS region must be set in the config."); + throw new IllegalArgumentException("The AWS region ('" + KinesisConfigConstants.CONFIG_AWS_REGION + "') must be set in the config."); } else { // specified AWS Region name must be recognizable if (!AWSUtil.isValidRegion(config.getProperty(KinesisConfigConstants.CONFIG_AWS_REGION))) { diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java index 9c9bd280fa8ee..39bb70ece5fcc 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java @@ -163,6 +163,8 @@ public void open(Configuration parameters) throws Exception { KinesisProducerConfiguration config = new KinesisProducerConfiguration(); config.setRegion(this.region); config.setCredentialsProvider(new StaticCredentialsProvider(new BasicAWSCredentials(this.accessKey, this.secretKey))); + //config.setCollectionMaxCount(1); + //config.setAggregationMaxCount(1); producer = new KinesisProducer(config); callback = new FutureCallback() { @Override diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 9bf8468e689d7..6aaad8a45a2dd 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -21,6 +21,7 @@ import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; +import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -105,7 +106,7 @@ public void run(SourceFunction.SourceContext sourceContext, ArrayList> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size()); for (Map.Entry assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) { ShardConsumerThread thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(), - assignedShard.getValue(), sourceContext, deserializationSchema, lastSequenceNums); + assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums); thread.setName(String.format("ShardConsumer - %s - %s/%s", taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId())); thread.setDaemon(true); From a8b5554bb369454007fe31469003e86ce1004be4 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Wed, 18 May 2016 23:31:33 +0800 Subject: [PATCH 31/70] [FLINK-3926] [core] Fix NumberFormatException in TupleTypeInfo.getFieldIndex() This closes #2004 --- .../apache/flink/api/java/typeutils/TupleTypeInfo.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java index 051ad0dfd309a..d525ffb6a6d95 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java @@ -84,11 +84,12 @@ public String[] getFieldNames() { @Override @PublicEvolving public int getFieldIndex(String fieldName) { - int fieldIndex = Integer.parseInt(fieldName.substring(1)); - if (fieldIndex >= getArity()) { - return -1; + for (int i = 0; i < fieldNames.length; i++) { + if (fieldNames[i].equals(fieldName)) { + return i; + } } - return fieldIndex; + return -1; } @SuppressWarnings("unchecked") From c65e1bf7729df70df6a7cd604ae86a329f97ae2b Mon Sep 17 00:00:00 2001 From: Mark Reddy Date: Tue, 17 May 2016 20:38:36 +0100 Subject: [PATCH 32/70] [FLINK-3914] [runtime] Fix race when attempting to create temp dir in BlobUtils. This closes #2000 --- .../src/main/java/org/apache/flink/runtime/blob/BlobUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java index 99237a37c0282..830269d05e6be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java @@ -108,7 +108,7 @@ static File initStorageDirectory(String storageDirectory) { static File getIncomingDirectory(File storageDir) { final File incomingDir = new File(storageDir, "incoming"); - if (!incomingDir.exists() && !incomingDir.mkdirs()) { + if (!incomingDir.mkdirs() && !incomingDir.exists()) { throw new RuntimeException("Cannot create directory for incoming files " + incomingDir.getAbsolutePath()); } From 0dabd4151bef84922bb59ffb1520e457d2bfa435 Mon Sep 17 00:00:00 2001 From: Flavio Pompermaier Date: Tue, 26 Apr 2016 19:10:53 +0200 Subject: [PATCH 33/70] [FLINK-3750] [jdbcConnector] Refactor JdbcInputFormat and JdbcOutputFormat. - New Input- and OutputFormat use Row instead of Tuple types to support null values. - JdbcInputFormat supports parallel input due to PreparedStatement and binding values for parameters. This closes #1941 --- flink-batch-connectors/flink-jdbc/pom.xml | 6 + .../api/java/io/jdbc/JDBCInputFormat.java | 392 ++++++++++-------- .../api/java/io/jdbc/JDBCOutputFormat.java | 255 +++++++----- .../split/GenericParameterValuesProvider.java | 44 ++ .../NumericBetweenParametersProvider.java | 66 +++ .../jdbc/split/ParameterValuesProvider.java} | 24 +- .../flink/api/java/io/jdbc/JDBCFullTest.java | 101 +++++ .../api/java/io/jdbc/JDBCInputFormatTest.java | 297 +++++++------ .../java/io/jdbc/JDBCOutputFormatTest.java | 222 ++++------ .../flink/api/java/io/jdbc/JDBCTestBase.java | 183 ++++++++ .../api/java/io/jdbc/example/JDBCExample.java | 102 ----- 11 files changed, 1021 insertions(+), 671 deletions(-) create mode 100644 flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java create mode 100644 flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java rename flink-batch-connectors/flink-jdbc/src/{test/java/org/apache/flink/api/java/io/jdbc/DerbyUtil.java => main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java} (58%) create mode 100644 flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java create mode 100644 flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java delete mode 100644 flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/example/JDBCExample.java diff --git a/flink-batch-connectors/flink-jdbc/pom.xml b/flink-batch-connectors/flink-jdbc/pom.xml index efa3d2fa08eb4..cb169ea0afd92 100644 --- a/flink-batch-connectors/flink-jdbc/pom.xml +++ b/flink-batch-connectors/flink-jdbc/pom.xml @@ -36,6 +36,12 @@ under the License. jar + + org.apache.flink + flink-table_2.10 + ${project.version} + provided + org.apache.flink flink-java diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index b7643508f17d5..b4246f5a9c7bb 100644 --- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -19,114 +19,236 @@ package org.apache.flink.api.java.io.jdbc; import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Array; import java.sql.Connection; +import java.sql.Date; import java.sql.DriverManager; +import java.sql.PreparedStatement; import java.sql.ResultSet; -import java.sql.ResultSetMetaData; import java.sql.SQLException; -import java.sql.Statement; - -import org.apache.flink.api.common.io.NonParallelInput; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Arrays; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.table.Row; +import org.apache.flink.api.table.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.types.NullValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * InputFormat to read data from a database and generate tuples. + * InputFormat to read data from a database and generate Rows. * The InputFormat has to be configured using the supplied InputFormatBuilder. - * - * @param - * @see Tuple + * A valid RowTypeInfo must be properly configured in the builder, e.g.:
+ * + *

+ * TypeInformation[] fieldTypes = new TypeInformation[] {
+ *		BasicTypeInfo.INT_TYPE_INFO,
+ *		BasicTypeInfo.STRING_TYPE_INFO,
+ *		BasicTypeInfo.STRING_TYPE_INFO,
+ *		BasicTypeInfo.DOUBLE_TYPE_INFO,
+ *		BasicTypeInfo.INT_TYPE_INFO
+ *	};
+ *
+ * RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes);
+ *
+ * JDBCInputFormat jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+ *				.setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+ *				.setDBUrl("jdbc:derby:memory:ebookshop")
+ *				.setQuery("select * from books")
+ *				.setRowTypeInfo(rowTypeInfo)
+ *				.finish();
+ * 
+ * + * In order to query the JDBC source in parallel, you need to provide a + * parameterized query template (i.e. a valid {@link PreparedStatement}) and + * a {@link ParameterValuesProvider} which provides binding values for the + * query parameters. E.g.:
+ * + *

+ *
+ * Serializable[][] queryParameters = new String[2][1];
+ * queryParameters[0] = new String[]{"Kumar"};
+ * queryParameters[1] = new String[]{"Tan Ah Teck"};
+ *
+ * JDBCInputFormat jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat()
+ *				.setDrivername("org.apache.derby.jdbc.EmbeddedDriver")
+ *				.setDBUrl("jdbc:derby:memory:ebookshop")
+ *				.setQuery("select * from books WHERE author = ?")
+ *				.setRowTypeInfo(rowTypeInfo)
+ *				.setParametersProvider(new GenericParameterValuesProvider(queryParameters))
+ *				.finish();
+ * 
+ * + * @see Row + * @see ParameterValuesProvider + * @see PreparedStatement * @see DriverManager */ -public class JDBCInputFormat extends RichInputFormat implements NonParallelInput { - private static final long serialVersionUID = 1L; +public class JDBCInputFormat extends RichInputFormat implements ResultTypeQueryable { + private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class); private String username; private String password; private String drivername; private String dbURL; - private String query; + private String queryTemplate; private int resultSetType; private int resultSetConcurrency; + private RowTypeInfo rowTypeInfo; private transient Connection dbConn; - private transient Statement statement; + private transient PreparedStatement statement; private transient ResultSet resultSet; - private int[] columnTypes = null; + private boolean hasNext; + private Object[][] parameterValues; public JDBCInputFormat() { } + @Override + public RowTypeInfo getProducedType() { + return rowTypeInfo; + } + @Override public void configure(Configuration parameters) { + //do nothing here } - /** - * Connects to the source database and executes the query. - * - * @param ignored - * @throws IOException - */ @Override - public void open(InputSplit ignored) throws IOException { + public void openInputFormat() { + //called once per inputFormat (on open) try { - establishConnection(); - statement = dbConn.createStatement(resultSetType, resultSetConcurrency); - resultSet = statement.executeQuery(query); + Class.forName(drivername); + if (username == null) { + dbConn = DriverManager.getConnection(dbURL); + } else { + dbConn = DriverManager.getConnection(dbURL, username, password); + } + statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency); } catch (SQLException se) { - close(); throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } catch (ClassNotFoundException cnfe) { throw new IllegalArgumentException("JDBC-Class not found. - " + cnfe.getMessage(), cnfe); } } - private void establishConnection() throws SQLException, ClassNotFoundException { - Class.forName(drivername); - if (username == null) { - dbConn = DriverManager.getConnection(dbURL); - } else { - dbConn = DriverManager.getConnection(dbURL, username, password); + @Override + public void closeInputFormat() { + //called once per inputFormat (on close) + try { + if(statement != null) { + statement.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat Statement couldn't be closed - " + se.getMessage()); + } finally { + statement = null; + } + + try { + if(dbConn != null) { + dbConn.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + dbConn = null; } + + parameterValues = null; } /** - * Closes all resources used. + * Connects to the source database and executes the query in a parallel + * fashion if + * this {@link InputFormat} is built using a parameterized query (i.e. using + * a {@link PreparedStatement}) + * and a proper {@link ParameterValuesProvider}, in a non-parallel + * fashion otherwise. * - * @throws IOException Indicates that a resource could not be closed. + * @param inputSplit which is ignored if this InputFormat is executed as a + * non-parallel source, + * a "hook" to the query parameters otherwise (using its + * splitNumber) + * @throws IOException if there's an error during the execution of the query */ @Override - public void close() throws IOException { + public void open(InputSplit inputSplit) throws IOException { try { - resultSet.close(); + if (inputSplit != null && parameterValues != null) { + for (int i = 0; i < parameterValues[inputSplit.getSplitNumber()].length; i++) { + Object param = parameterValues[inputSplit.getSplitNumber()][i]; + if (param instanceof String) { + statement.setString(i + 1, (String) param); + } else if (param instanceof Long) { + statement.setLong(i + 1, (Long) param); + } else if (param instanceof Integer) { + statement.setInt(i + 1, (Integer) param); + } else if (param instanceof Double) { + statement.setDouble(i + 1, (Double) param); + } else if (param instanceof Boolean) { + statement.setBoolean(i + 1, (Boolean) param); + } else if (param instanceof Float) { + statement.setFloat(i + 1, (Float) param); + } else if (param instanceof BigDecimal) { + statement.setBigDecimal(i + 1, (BigDecimal) param); + } else if (param instanceof Byte) { + statement.setByte(i + 1, (Byte) param); + } else if (param instanceof Short) { + statement.setShort(i + 1, (Short) param); + } else if (param instanceof Date) { + statement.setDate(i + 1, (Date) param); + } else if (param instanceof Time) { + statement.setTime(i + 1, (Time) param); + } else if (param instanceof Timestamp) { + statement.setTimestamp(i + 1, (Timestamp) param); + } else if (param instanceof Array) { + statement.setArray(i + 1, (Array) param); + } else { + //extends with other types if needed + throw new IllegalArgumentException("open() failed. Parameter " + i + " of type " + param.getClass() + " is not handled (yet)." ); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Executing '%s' with parameters %s", queryTemplate, Arrays.deepToString(parameterValues[inputSplit.getSplitNumber()]))); + } + } + resultSet = statement.executeQuery(); + hasNext = resultSet.next(); } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } catch (NullPointerException npe) { + throw new IllegalArgumentException("open() failed." + se.getMessage(), se); } - try { - statement.close(); - } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } catch (NullPointerException npe) { + } + + /** + * Closes all resources used. + * + * @throws IOException Indicates that a resource could not be closed. + */ + @Override + public void close() throws IOException { + if(resultSet == null) { + return; } try { - dbConn.close(); + resultSet.close(); } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } catch (NullPointerException npe) { + LOG.info("Inputformat ResultSet couldn't be closed - " + se.getMessage()); } } @@ -138,150 +260,35 @@ public void close() throws IOException { */ @Override public boolean reachedEnd() throws IOException { - try { - if (resultSet.isLast()) { - close(); - return true; - } - return false; - } catch (SQLException se) { - throw new IOException("Couldn't evaluate reachedEnd() - " + se.getMessage(), se); - } + return !hasNext; } /** * Stores the next resultSet row in a tuple * - * @param tuple - * @return tuple containing next row + * @param row row to be reused. + * @return row containing next {@link Row} * @throws java.io.IOException */ @Override - public OUT nextRecord(OUT tuple) throws IOException { + public Row nextRecord(Row row) throws IOException { try { - resultSet.next(); - if (columnTypes == null) { - extractTypes(tuple); + if (!hasNext) { + return null; } - addValue(tuple); - return tuple; + for (int pos = 0; pos < row.productArity(); pos++) { + row.setField(pos, resultSet.getObject(pos + 1)); + } + //update hasNext after we've read the record + hasNext = resultSet.next(); + return row; } catch (SQLException se) { - close(); throw new IOException("Couldn't read data - " + se.getMessage(), se); } catch (NullPointerException npe) { - close(); throw new IOException("Couldn't access resultSet", npe); } } - private void extractTypes(OUT tuple) throws SQLException, IOException { - ResultSetMetaData resultSetMetaData = resultSet.getMetaData(); - columnTypes = new int[resultSetMetaData.getColumnCount()]; - if (tuple.getArity() != columnTypes.length) { - close(); - throw new IOException("Tuple size does not match columncount"); - } - for (int pos = 0; pos < columnTypes.length; pos++) { - columnTypes[pos] = resultSetMetaData.getColumnType(pos + 1); - } - } - - /** - * Enters data value from the current resultSet into a Record. - * - * @param reuse Target Record. - */ - private void addValue(OUT reuse) throws IOException, SQLException { - for (int pos = 0; pos < columnTypes.length; pos++) { - try { - switch (columnTypes[pos]) { - case java.sql.Types.NULL: - reuse.setField(NullValue.getInstance(), pos); - break; - case java.sql.Types.BOOLEAN: - reuse.setField(resultSet.getBoolean(pos + 1), pos); - break; - case java.sql.Types.BIT: - reuse.setField(resultSet.getBoolean(pos + 1), pos); - break; - case java.sql.Types.CHAR: - reuse.setField(resultSet.getString(pos + 1), pos); - break; - case java.sql.Types.NCHAR: - reuse.setField(resultSet.getString(pos + 1), pos); - break; - case java.sql.Types.VARCHAR: - reuse.setField(resultSet.getString(pos + 1), pos); - break; - case java.sql.Types.LONGVARCHAR: - reuse.setField(resultSet.getString(pos + 1), pos); - break; - case java.sql.Types.LONGNVARCHAR: - reuse.setField(resultSet.getString(pos + 1), pos); - break; - case java.sql.Types.TINYINT: - reuse.setField(resultSet.getShort(pos + 1), pos); - break; - case java.sql.Types.SMALLINT: - reuse.setField(resultSet.getShort(pos + 1), pos); - break; - case java.sql.Types.BIGINT: - reuse.setField(resultSet.getLong(pos + 1), pos); - break; - case java.sql.Types.INTEGER: - reuse.setField(resultSet.getInt(pos + 1), pos); - break; - case java.sql.Types.FLOAT: - reuse.setField(resultSet.getDouble(pos + 1), pos); - break; - case java.sql.Types.REAL: - reuse.setField(resultSet.getFloat(pos + 1), pos); - break; - case java.sql.Types.DOUBLE: - reuse.setField(resultSet.getDouble(pos + 1), pos); - break; - case java.sql.Types.DECIMAL: - reuse.setField(resultSet.getBigDecimal(pos + 1).doubleValue(), pos); - break; - case java.sql.Types.NUMERIC: - reuse.setField(resultSet.getBigDecimal(pos + 1).doubleValue(), pos); - break; - case java.sql.Types.DATE: - reuse.setField(resultSet.getDate(pos + 1).toString(), pos); - break; - case java.sql.Types.TIME: - reuse.setField(resultSet.getTime(pos + 1).getTime(), pos); - break; - case java.sql.Types.TIMESTAMP: - reuse.setField(resultSet.getTimestamp(pos + 1).toString(), pos); - break; - case java.sql.Types.SQLXML: - reuse.setField(resultSet.getSQLXML(pos + 1).toString(), pos); - break; - default: - throw new SQLException("Unsupported sql-type [" + columnTypes[pos] + "] on column [" + pos + "]"); - - // case java.sql.Types.BINARY: - // case java.sql.Types.VARBINARY: - // case java.sql.Types.LONGVARBINARY: - // case java.sql.Types.ARRAY: - // case java.sql.Types.JAVA_OBJECT: - // case java.sql.Types.BLOB: - // case java.sql.Types.CLOB: - // case java.sql.Types.NCLOB: - // case java.sql.Types.DATALINK: - // case java.sql.Types.DISTINCT: - // case java.sql.Types.OTHER: - // case java.sql.Types.REF: - // case java.sql.Types.ROWID: - // case java.sql.Types.STRUCT: - } - } catch (NullPointerException npe) { - throw new IOException("Encountered null value for column " + pos + ". Decimal, Numeric, Date, Time, Timestamp and SQLXML columns may not contain NULL values."); - } - } - } - @Override public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { return cachedStatistics; @@ -289,17 +296,20 @@ public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOEx @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { - GenericInputSplit[] split = { - new GenericInputSplit(0, 1) - }; - return split; + if (parameterValues == null) { + return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; + } + GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length]; + for (int i = 0; i < ret.length; i++) { + ret[i] = new GenericInputSplit(i, ret.length); + } + return ret; } @Override public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) { return new DefaultInputSplitAssigner(inputSplits); } - /** * A builder used to set parameters to the output format's configuration in a fluent way. @@ -314,6 +324,7 @@ public static class JDBCInputFormatBuilder { public JDBCInputFormatBuilder() { this.format = new JDBCInputFormat(); + //using TYPE_FORWARD_ONLY for high performance reads this.format.resultSetType = ResultSet.TYPE_FORWARD_ONLY; this.format.resultSetConcurrency = ResultSet.CONCUR_READ_ONLY; } @@ -339,7 +350,7 @@ public JDBCInputFormatBuilder setDBUrl(String dbURL) { } public JDBCInputFormatBuilder setQuery(String query) { - format.query = query; + format.queryTemplate = query; return this; } @@ -353,6 +364,16 @@ public JDBCInputFormatBuilder setResultSetConcurrency(int resultSetConcurrency) return this; } + public JDBCInputFormatBuilder setParametersProvider(ParameterValuesProvider parameterValuesProvider) { + format.parameterValues = parameterValuesProvider.getParameterValues(); + return this; + } + + public JDBCInputFormatBuilder setRowTypeInfo(RowTypeInfo rowTypeInfo) { + format.rowTypeInfo = rowTypeInfo; + return this; + } + public JDBCInputFormat finish() { if (format.username == null) { LOG.info("Username was not supplied separately."); @@ -363,14 +384,21 @@ public JDBCInputFormat finish() { if (format.dbURL == null) { throw new IllegalArgumentException("No database URL supplied"); } - if (format.query == null) { + if (format.queryTemplate == null) { throw new IllegalArgumentException("No query supplied"); } if (format.drivername == null) { throw new IllegalArgumentException("No driver supplied"); } + if (format.rowTypeInfo == null) { + throw new IllegalArgumentException("No " + RowTypeInfo.class.getSimpleName() + " supplied"); + } + if (format.parameterValues == null) { + LOG.debug("No input splitting configured (data will be read with parallelism 1)."); + } return format; } + } } diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 614c5b71cfb6f..5464a941bb8c6 100644 --- a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -24,47 +24,46 @@ import java.sql.PreparedStatement; import java.sql.SQLException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.table.Row; import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * OutputFormat to write tuples into a database. * The OutputFormat has to be configured using the supplied OutputFormatBuilder. * - * @param * @see Tuple * @see DriverManager */ -public class JDBCOutputFormat extends RichOutputFormat { +public class JDBCOutputFormat extends RichOutputFormat { private static final long serialVersionUID = 1L; - - @SuppressWarnings("unused") + private static final Logger LOG = LoggerFactory.getLogger(JDBCOutputFormat.class); - + private String username; private String password; private String drivername; private String dbURL; private String query; private int batchInterval = 5000; - + private Connection dbConn; private PreparedStatement upload; - - private SupportedTypes[] types = null; - + private int batchCount = 0; - + + public int[] typesArray; + public JDBCOutputFormat() { } - + @Override public void configure(Configuration parameters) { } - + /** * Connects to the target database and initializes the prepared statement. * @@ -78,14 +77,12 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); upload = dbConn.prepareStatement(query); } catch (SQLException sqe) { - close(); - throw new IllegalArgumentException("open() failed:\t!", sqe); + throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { - close(); - throw new IllegalArgumentException("JDBC-Class not found:\t", cnfe); + throw new IllegalArgumentException("JDBC driver class not found.", cnfe); } } - + private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); if (username == null) { @@ -94,86 +91,125 @@ private void establishConnection() throws SQLException, ClassNotFoundException { dbConn = DriverManager.getConnection(dbURL, username, password); } } - - private enum SupportedTypes { - BOOLEAN, - BYTE, - SHORT, - INTEGER, - LONG, - STRING, - FLOAT, - DOUBLE - } - + /** * Adds a record to the prepared statement. *

* When this method is called, the output format is guaranteed to be opened. + *

+ * + * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to + * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) * - * @param tuple The records to add to the output. + * @param row The records to add to the output. + * @see PreparedStatement * @throws IOException Thrown, if the records could not be added due to an I/O problem. */ @Override - public void writeRecord(OUT tuple) throws IOException { + public void writeRecord(Row row) throws IOException { + + if (typesArray != null && typesArray.length > 0 && typesArray.length == row.productArity()) { + LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); + } try { - if (types == null) { - extractTypes(tuple); + + if (typesArray == null ) { + // no types provided + for (int index = 0; index < row.productArity(); index++) { + LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.productElement(index)); + upload.setObject(index + 1, row.productElement(index)); + } + } else { + // types provided + for (int index = 0; index < row.productArity(); index++) { + + if (row.productElement(index) == null) { + upload.setNull(index + 1, typesArray[index]); + } else { + // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html + switch (typesArray[index]) { + case java.sql.Types.NULL: + upload.setNull(index + 1, typesArray[index]); + break; + case java.sql.Types.BOOLEAN: + case java.sql.Types.BIT: + upload.setBoolean(index + 1, (boolean) row.productElement(index)); + break; + case java.sql.Types.CHAR: + case java.sql.Types.NCHAR: + case java.sql.Types.VARCHAR: + case java.sql.Types.LONGVARCHAR: + case java.sql.Types.LONGNVARCHAR: + upload.setString(index + 1, (String) row.productElement(index)); + break; + case java.sql.Types.TINYINT: + upload.setByte(index + 1, (byte) row.productElement(index)); + break; + case java.sql.Types.SMALLINT: + upload.setShort(index + 1, (short) row.productElement(index)); + break; + case java.sql.Types.INTEGER: + upload.setInt(index + 1, (int) row.productElement(index)); + break; + case java.sql.Types.BIGINT: + upload.setLong(index + 1, (long) row.productElement(index)); + break; + case java.sql.Types.REAL: + upload.setFloat(index + 1, (float) row.productElement(index)); + break; + case java.sql.Types.FLOAT: + case java.sql.Types.DOUBLE: + upload.setDouble(index + 1, (double) row.productElement(index)); + break; + case java.sql.Types.DECIMAL: + case java.sql.Types.NUMERIC: + upload.setBigDecimal(index + 1, (java.math.BigDecimal) row.productElement(index)); + break; + case java.sql.Types.DATE: + upload.setDate(index + 1, (java.sql.Date) row.productElement(index)); + break; + case java.sql.Types.TIME: + upload.setTime(index + 1, (java.sql.Time) row.productElement(index)); + break; + case java.sql.Types.TIMESTAMP: + upload.setTimestamp(index + 1, (java.sql.Timestamp) row.productElement(index)); + break; + case java.sql.Types.BINARY: + case java.sql.Types.VARBINARY: + case java.sql.Types.LONGVARBINARY: + upload.setBytes(index + 1, (byte[]) row.productElement(index)); + break; + default: + upload.setObject(index + 1, row.productElement(index)); + LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", + typesArray[index], index + 1, row.productElement(index)); + // case java.sql.Types.SQLXML + // case java.sql.Types.ARRAY: + // case java.sql.Types.JAVA_OBJECT: + // case java.sql.Types.BLOB: + // case java.sql.Types.CLOB: + // case java.sql.Types.NCLOB: + // case java.sql.Types.DATALINK: + // case java.sql.Types.DISTINCT: + // case java.sql.Types.OTHER: + // case java.sql.Types.REF: + // case java.sql.Types.ROWID: + // case java.sql.Types.STRUC + } + } + } } - addValues(tuple); upload.addBatch(); batchCount++; if (batchCount >= batchInterval) { upload.executeBatch(); batchCount = 0; } - } catch (SQLException sqe) { - close(); - throw new IllegalArgumentException("writeRecord() failed", sqe); - } catch (IllegalArgumentException iae) { - close(); - throw new IllegalArgumentException("writeRecord() failed", iae); - } - } - - private void extractTypes(OUT tuple) { - types = new SupportedTypes[tuple.getArity()]; - for (int x = 0; x < tuple.getArity(); x++) { - types[x] = SupportedTypes.valueOf(tuple.getField(x).getClass().getSimpleName().toUpperCase()); + } catch (SQLException | IllegalArgumentException e) { + throw new IllegalArgumentException("writeRecord() failed", e); } } - - private void addValues(OUT tuple) throws SQLException { - for (int index = 0; index < tuple.getArity(); index++) { - switch (types[index]) { - case BOOLEAN: - upload.setBoolean(index + 1, (Boolean) tuple.getField(index)); - break; - case BYTE: - upload.setByte(index + 1, (Byte) tuple.getField(index)); - break; - case SHORT: - upload.setShort(index + 1, (Short) tuple.getField(index)); - break; - case INTEGER: - upload.setInt(index + 1, (Integer) tuple.getField(index)); - break; - case LONG: - upload.setLong(index + 1, (Long) tuple.getField(index)); - break; - case STRING: - upload.setString(index + 1, (String) tuple.getField(index)); - break; - case FLOAT: - upload.setFloat(index + 1, (Float) tuple.getField(index)); - break; - case DOUBLE: - upload.setDouble(index + 1, (Double) tuple.getField(index)); - break; - } - } - } - + /** * Executes prepared statement and closes all resources of this instance. * @@ -182,70 +218,78 @@ private void addValues(OUT tuple) throws SQLException { @Override public void close() throws IOException { try { - upload.executeBatch(); - batchCount = 0; - } catch (SQLException se) { - throw new IllegalArgumentException("close() failed", se); - } catch (NullPointerException se) { - } - try { - upload.close(); + if (upload != null) { + upload.executeBatch(); + upload.close(); + } } catch (SQLException se) { LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } catch (NullPointerException npe) { + } finally { + upload = null; + batchCount = 0; } + try { - dbConn.close(); + if (dbConn != null) { + dbConn.close(); + } } catch (SQLException se) { LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } catch (NullPointerException npe) { + } finally { + dbConn = null; } } - + public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { return new JDBCOutputFormatBuilder(); } - + public static class JDBCOutputFormatBuilder { private final JDBCOutputFormat format; - + protected JDBCOutputFormatBuilder() { this.format = new JDBCOutputFormat(); } - + public JDBCOutputFormatBuilder setUsername(String username) { format.username = username; return this; } - + public JDBCOutputFormatBuilder setPassword(String password) { format.password = password; return this; } - + public JDBCOutputFormatBuilder setDrivername(String drivername) { format.drivername = drivername; return this; } - + public JDBCOutputFormatBuilder setDBUrl(String dbURL) { format.dbURL = dbURL; return this; } - + public JDBCOutputFormatBuilder setQuery(String query) { format.query = query; return this; } - + public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { format.batchInterval = batchInterval; return this; } - + + public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { + format.typesArray = typesArray; + return this; + } + /** - Finalizes the configuration and checks validity. - @return Configured JDBCOutputFormat + * Finalizes the configuration and checks validity. + * + * @return Configured JDBCOutputFormat */ public JDBCOutputFormat finish() { if (format.username == null) { @@ -263,8 +307,9 @@ public JDBCOutputFormat finish() { if (format.drivername == null) { throw new IllegalArgumentException("No driver supplied"); } + return format; } } - + } diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java new file mode 100644 index 0000000000000..6c70a8cbd85d9 --- /dev/null +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/GenericParameterValuesProvider.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.java.io.jdbc.split; + +import java.io.Serializable; + +import org.apache.flink.api.java.io.jdbc.JDBCInputFormat; + +/** + * + * This splits generator actually does nothing but wrapping the query parameters + * computed by the user before creating the {@link JDBCInputFormat} instance. + * + * */ +public class GenericParameterValuesProvider implements ParameterValuesProvider { + + private final Serializable[][] parameters; + + public GenericParameterValuesProvider(Serializable[][] parameters) { + this.parameters = parameters; + } + + @Override + public Serializable[][] getParameterValues(){ + //do nothing...precomputed externally + return parameters; + } + +} diff --git a/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java new file mode 100644 index 0000000000000..306663e82dde7 --- /dev/null +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/NumericBetweenParametersProvider.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.java.io.jdbc.split; + +import java.io.Serializable; + +/** + * + * This query generator assumes that the query to parameterize contains a BETWEEN constraint on a numeric column. + * The generated query set will be of size equal to the configured fetchSize (apart the last one range), + * ranging from the min value up to the max. + * + * For example, if there's a table BOOKS with a numeric PK id, using a query like: + *

+ *   SELECT * FROM BOOKS WHERE id BETWEEN ? AND ?
+ * 
+ * + * you can use this class to automatically generate the parameters of the BETWEEN clause, + * based on the passed constructor parameters. + * + * */ +public class NumericBetweenParametersProvider implements ParameterValuesProvider { + + private long fetchSize; + private final long min; + private final long max; + + public NumericBetweenParametersProvider(long fetchSize, long min, long max) { + this.fetchSize = fetchSize; + this.min = min; + this.max = max; + } + + @Override + public Serializable[][] getParameterValues(){ + double maxElemCount = (max - min) + 1; + int size = new Double(Math.ceil(maxElemCount / fetchSize)).intValue(); + Serializable[][] parameters = new Serializable[size][2]; + int count = 0; + for (long i = min; i < max; i += fetchSize, count++) { + long currentLimit = i + fetchSize - 1; + parameters[count] = new Long[]{i,currentLimit}; + if (currentLimit + 1 + fetchSize > max) { + parameters[count + 1] = new Long[]{currentLimit + 1, max}; + break; + } + } + return parameters; + } + +} diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyUtil.java b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java similarity index 58% rename from flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyUtil.java rename to flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java index 523b8b56ca868..6c632f869d0d6 100644 --- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyUtil.java +++ b/flink-batch-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/split/ParameterValuesProvider.java @@ -15,17 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.java.io.jdbc; +package org.apache.flink.api.java.io.jdbc.split; -import java.io.OutputStream; +import java.io.Serializable; + +import org.apache.flink.api.java.io.jdbc.JDBCInputFormat; -@SuppressWarnings("unused") /** - * Utility class to disable derby logging - */ -public class DerbyUtil { - public static final OutputStream DEV_NULL = new OutputStream() { - public void write(int b) { - } - }; + * + * This interface is used by the {@link JDBCInputFormat} to compute the list of parallel query to run (i.e. splits). + * Each query will be parameterized using a row of the matrix provided by each {@link ParameterValuesProvider} implementation + * + * */ +public interface ParameterValuesProvider { + + /** Returns the necessary parameters array to use for query in parallel a table */ + public Serializable[][] getParameterValues(); + } diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java new file mode 100644 index 0000000000000..da9469b2f6541 --- /dev/null +++ b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.java.io.jdbc; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.jdbc.JDBCInputFormat.JDBCInputFormatBuilder; +import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; +import org.apache.flink.api.table.Row; +import org.junit.Assert; +import org.junit.Test; + +public class JDBCFullTest extends JDBCTestBase { + + @Test + public void testJdbcInOut() throws Exception { + //run without parallelism + runTest(false); + + //cleanup + JDBCTestBase.tearDownClass(); + JDBCTestBase.prepareTestDb(); + + //run expliting parallelism + runTest(true); + + } + + private void runTest(boolean exploitParallelism) { + ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment(); + JDBCInputFormatBuilder inputBuilder = JDBCInputFormat.buildJDBCInputFormat() + .setDrivername(JDBCTestBase.DRIVER_CLASS) + .setDBUrl(JDBCTestBase.DB_URL) + .setQuery(JDBCTestBase.SELECT_ALL_BOOKS) + .setRowTypeInfo(rowTypeInfo); + + if(exploitParallelism) { + final int fetchSize = 1; + final Long min = new Long(JDBCTestBase.testData[0][0].toString()); + final Long max = new Long(JDBCTestBase.testData[JDBCTestBase.testData.length - fetchSize][0].toString()); + //use a "splittable" query to exploit parallelism + inputBuilder = inputBuilder + .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setParametersProvider(new NumericBetweenParametersProvider(fetchSize, min, max)); + } + DataSet source = environment.createInput(inputBuilder.finish()); + + //NOTE: in this case (with Derby driver) setSqlTypes could be skipped, but + //some database, doens't handle correctly null values when no column type specified + //in PreparedStatement.setObject (see its javadoc for more details) + source.output(JDBCOutputFormat.buildJDBCOutputFormat() + .setDrivername(JDBCTestBase.DRIVER_CLASS) + .setDBUrl(JDBCTestBase.DB_URL) + .setQuery("insert into newbooks (id,title,author,price,qty) values (?,?,?,?,?)") + .setSqlTypes(new int[]{Types.INTEGER, Types.VARCHAR, Types.VARCHAR,Types.DOUBLE,Types.INTEGER}) + .finish()); + try { + environment.execute(); + } catch (Exception e) { + Assert.fail("JDBC full test failed. " + e.getMessage()); + } + + try ( + Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL); + PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS); + ResultSet resultSet = statement.executeQuery() + ) { + int count = 0; + while (resultSet.next()) { + count++; + } + Assert.assertEquals(JDBCTestBase.testData.length, count); + } catch (SQLException e) { + Assert.fail("JDBC full test failed. " + e.getMessage()); + } + } + +} diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java index 3fb0278c4ed31..efae076c3d2a0 100644 --- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java +++ b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java @@ -19,180 +19,229 @@ package org.apache.flink.api.java.io.jdbc; import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; +import java.io.Serializable; import java.sql.ResultSet; - -import org.junit.Assert; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.io.jdbc.split.GenericParameterValuesProvider; +import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; +import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; +import org.apache.flink.api.table.Row; +import org.apache.flink.core.io.InputSplit; import org.junit.After; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.Assert; import org.junit.Test; -public class JDBCInputFormatTest { - JDBCInputFormat jdbcInputFormat; +public class JDBCInputFormatTest extends JDBCTestBase { - static Connection conn; - - static final Object[][] dbData = { - {1001, ("Java for dummies"), ("Tan Ah Teck"), 11.11, 11}, - {1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22}, - {1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33}, - {1004, ("A Cup of Java"), ("Kumar"), 44.44, 44}, - {1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55}}; - - @BeforeClass - public static void setUpClass() { - try { - prepareDerbyDatabase(); - } catch (Exception e) { - Assert.fail(); - } - } + private JDBCInputFormat jdbcInputFormat; - private static void prepareDerbyDatabase() throws ClassNotFoundException, SQLException { - System.setProperty("derby.stream.error.field", "org.apache.flink.api.java.io.jdbc.DerbyUtil.DEV_NULL"); - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - createTable(); - insertDataToSQLTable(); - conn.close(); - } - - private static void createTable() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE books ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - } - - private static void insertDataToSQLTable() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', 'Tan Ah Teck', 11.11, 11),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', 'Tan Ah Teck', 22.22, 22),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', 'Mohammad Ali', 33.33, 33),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', 'Kumar', 44.44, 44),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', 'Kevin Jones', 55.55, 55)"); - - Statement stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - } - - @AfterClass - public static void tearDownClass() { - cleanUpDerbyDatabases(); - } - - private static void cleanUpDerbyDatabases() { - try { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - - conn = DriverManager.getConnection(dbURL); - Statement stat = conn.createStatement(); - stat.executeUpdate("DROP TABLE books"); - stat.close(); - conn.close(); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(); + @After + public void tearDown() throws IOException { + if (jdbcInputFormat != null) { + jdbcInputFormat.close(); } + jdbcInputFormat = null; } - @After - public void tearDown() { - jdbcInputFormat = null; + @Test(expected = IllegalArgumentException.class) + public void testUntypedRowInfo() throws IOException { + jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() + .setDrivername("org.apache.derby.jdbc.idontexist") + .setDBUrl(DB_URL) + .setQuery(SELECT_ALL_BOOKS) + .finish(); + jdbcInputFormat.openInputFormat(); } @Test(expected = IllegalArgumentException.class) public void testInvalidDriver() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("select * from books") + .setDBUrl(DB_URL) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(rowTypeInfo) .finish(); - jdbcInputFormat.open(null); + jdbcInputFormat.openInputFormat(); } @Test(expected = IllegalArgumentException.class) public void testInvalidURL() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") + .setDrivername(DRIVER_CLASS) .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") - .setQuery("select * from books") + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(rowTypeInfo) .finish(); - jdbcInputFormat.open(null); + jdbcInputFormat.openInputFormat(); } @Test(expected = IllegalArgumentException.class) public void testInvalidQuery() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) .setQuery("iamnotsql") + .setRowTypeInfo(rowTypeInfo) .finish(); - jdbcInputFormat.open(null); + jdbcInputFormat.openInputFormat(); } @Test(expected = IllegalArgumentException.class) public void testIncompleteConfiguration() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setQuery("select * from books") - .finish(); - } - - @Test(expected = IOException.class) - public void testIncompatibleTuple() throws IOException { - jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("select * from books") + .setDrivername(DRIVER_CLASS) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(rowTypeInfo) .finish(); - jdbcInputFormat.open(null); - jdbcInputFormat.nextRecord(new Tuple2()); } @Test - public void testJDBCInputFormat() throws IOException { + public void testJDBCInputFormatWithoutParallelism() throws IOException, InstantiationException, IllegalAccessException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("select * from books") + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(rowTypeInfo) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) .finish(); + //this query does not exploit parallelism + Assert.assertEquals(1, jdbcInputFormat.createInputSplits(1).length); + jdbcInputFormat.openInputFormat(); jdbcInputFormat.open(null); - Tuple5 tuple = new Tuple5(); + Row row = new Row(5); int recordCount = 0; while (!jdbcInputFormat.reachedEnd()) { - jdbcInputFormat.nextRecord(tuple); - Assert.assertEquals("Field 0 should be int", Integer.class, tuple.getField(0).getClass()); - Assert.assertEquals("Field 1 should be String", String.class, tuple.getField(1).getClass()); - Assert.assertEquals("Field 2 should be String", String.class, tuple.getField(2).getClass()); - Assert.assertEquals("Field 3 should be float", Double.class, tuple.getField(3).getClass()); - Assert.assertEquals("Field 4 should be int", Integer.class, tuple.getField(4).getClass()); + Row next = jdbcInputFormat.nextRecord(row); + if (next == null) { + break; + } + + if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());} + if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());} + if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());} + if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());} + if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());} for (int x = 0; x < 5; x++) { - Assert.assertEquals(dbData[recordCount][x], tuple.getField(x)); + if(testData[recordCount][x]!=null) { + Assert.assertEquals(testData[recordCount][x], next.productElement(x)); + } } recordCount++; } - Assert.assertEquals(5, recordCount); + jdbcInputFormat.close(); + jdbcInputFormat.closeInputFormat(); + Assert.assertEquals(testData.length, recordCount); + } + + @Test + public void testJDBCInputFormatWithParallelismAndNumericColumnSplitting() throws IOException, InstantiationException, IllegalAccessException { + final int fetchSize = 1; + final Long min = new Long(JDBCTestBase.testData[0][0] + ""); + final Long max = new Long(JDBCTestBase.testData[JDBCTestBase.testData.length - fetchSize][0] + ""); + ParameterValuesProvider pramProvider = new NumericBetweenParametersProvider(fetchSize, min, max); + jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setRowTypeInfo(rowTypeInfo) + .setParametersProvider(pramProvider) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .finish(); + + jdbcInputFormat.openInputFormat(); + InputSplit[] splits = jdbcInputFormat.createInputSplits(1); + //this query exploit parallelism (1 split for every id) + Assert.assertEquals(testData.length, splits.length); + int recordCount = 0; + Row row = new Row(5); + for (int i = 0; i < splits.length; i++) { + jdbcInputFormat.open(splits[i]); + while (!jdbcInputFormat.reachedEnd()) { + Row next = jdbcInputFormat.nextRecord(row); + if (next == null) { + break; + } + if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());} + if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());} + if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());} + if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());} + if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());} + + for (int x = 0; x < 5; x++) { + if(testData[recordCount][x]!=null) { + Assert.assertEquals(testData[recordCount][x], next.productElement(x)); + } + } + recordCount++; + } + jdbcInputFormat.close(); + } + jdbcInputFormat.closeInputFormat(); + Assert.assertEquals(testData.length, recordCount); + } + + @Test + public void testJDBCInputFormatWithParallelismAndGenericSplitting() throws IOException, InstantiationException, IllegalAccessException { + Serializable[][] queryParameters = new String[2][1]; + queryParameters[0] = new String[]{"Kumar"}; + queryParameters[1] = new String[]{"Tan Ah Teck"}; + ParameterValuesProvider paramProvider = new GenericParameterValuesProvider(queryParameters); + jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR) + .setRowTypeInfo(rowTypeInfo) + .setParametersProvider(paramProvider) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .finish(); + jdbcInputFormat.openInputFormat(); + InputSplit[] splits = jdbcInputFormat.createInputSplits(1); + //this query exploit parallelism (1 split for every queryParameters row) + Assert.assertEquals(queryParameters.length, splits.length); + int recordCount = 0; + Row row = new Row(5); + for (int i = 0; i < splits.length; i++) { + jdbcInputFormat.open(splits[i]); + while (!jdbcInputFormat.reachedEnd()) { + Row next = jdbcInputFormat.nextRecord(row); + if (next == null) { + break; + } + if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());} + if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());} + if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());} + if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());} + if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());} + + recordCount++; + } + jdbcInputFormat.close(); + } + Assert.assertEquals(3, recordCount); + jdbcInputFormat.closeInputFormat(); + } + + @Test + public void testEmptyResults() throws IOException, InstantiationException, IllegalAccessException { + jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(SELECT_EMPTY) + .setRowTypeInfo(rowTypeInfo) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .finish(); + jdbcInputFormat.openInputFormat(); + jdbcInputFormat.open(null); + Row row = new Row(5); + int recordsCnt = 0; + while (!jdbcInputFormat.reachedEnd()) { + Assert.assertNull(jdbcInputFormat.nextRecord(row)); + recordsCnt++; + } + jdbcInputFormat.close(); + jdbcInputFormat.closeInputFormat(); + Assert.assertEquals(0, recordsCnt); } -} +} \ No newline at end of file diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java index 1031b9a702d94..086a84cfba20a 100644 --- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java +++ b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java @@ -21,105 +21,26 @@ import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; +import java.sql.PreparedStatement; import java.sql.ResultSet; - -import org.junit.Assert; +import java.sql.SQLException; import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.table.Row; import org.junit.After; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.Assert; import org.junit.Test; -public class JDBCOutputFormatTest { - private JDBCInputFormat jdbcInputFormat; - private JDBCOutputFormat jdbcOutputFormat; - - private static Connection conn; - - static final Object[][] dbData = { - {1001, ("Java for dummies"), ("Tan Ah Teck"), 11.11, 11}, - {1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22}, - {1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33}, - {1004, ("A Cup of Java"), ("Kumar"), 44.44, 44}, - {1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55}}; - - @BeforeClass - public static void setUpClass() throws SQLException { - try { - System.setProperty("derby.stream.error.field", "org.apache.flink.api.java.io.jdbc.DerbyUtil.DEV_NULL"); - prepareDerbyDatabase(); - } catch (ClassNotFoundException e) { - e.printStackTrace(); - Assert.fail(); - } - } - - private static void prepareDerbyDatabase() throws ClassNotFoundException, SQLException { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - conn = DriverManager.getConnection(dbURL); - createTable("books"); - createTable("newbooks"); - insertDataToSQLTables(); - conn.close(); - } - - private static void createTable(String tableName) throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE "); - sqlQueryBuilder.append(tableName); - sqlQueryBuilder.append(" ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - } - - private static void insertDataToSQLTables() throws SQLException { - StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', 'Tan Ah Teck', 11.11, 11),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', 'Tan Ah Teck', 22.22, 22),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', 'Mohammad Ali', 33.33, 33),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', 'Kumar', 44.44, 44),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', 'Kevin Jones', 55.55, 55)"); - - Statement stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - } - - @AfterClass - public static void tearDownClass() { - cleanUpDerbyDatabases(); - } +public class JDBCOutputFormatTest extends JDBCTestBase { - private static void cleanUpDerbyDatabases() { - try { - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - - conn = DriverManager.getConnection(dbURL); - Statement stat = conn.createStatement(); - stat.executeUpdate("DROP TABLE books"); - stat.executeUpdate("DROP TABLE newbooks"); - stat.close(); - conn.close(); - } catch (Exception e) { - e.printStackTrace(); - Assert.fail(); - } - } + private JDBCOutputFormat jdbcOutputFormat; + private Tuple5 tuple5 = new Tuple5<>(); @After - public void tearDown() { + public void tearDown() throws IOException { + if (jdbcOutputFormat != null) { + jdbcOutputFormat.close(); + } jdbcOutputFormat = null; } @@ -127,8 +48,8 @@ public void tearDown() { public void testInvalidDriver() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("insert into books (id, title, author, price, qty) values (?,?,?,?,?)") + .setDBUrl(DB_URL) + .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); } @@ -136,9 +57,9 @@ public void testInvalidDriver() throws IOException { @Test(expected = IllegalArgumentException.class) public void testInvalidURL() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") + .setDrivername(DRIVER_CLASS) .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") - .setQuery("insert into books (id, title, author, price, qty) values (?,?,?,?,?)") + .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); } @@ -146,8 +67,8 @@ public void testInvalidURL() throws IOException { @Test(expected = IllegalArgumentException.class) public void testInvalidQuery() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) .setQuery("iamnotsql") .finish(); jdbcOutputFormat.open(0, 1); @@ -156,8 +77,8 @@ public void testInvalidQuery() throws IOException { @Test(expected = IllegalArgumentException.class) public void testIncompleteConfiguration() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setQuery("insert into books (id, title, author, price, qty) values (?,?,?,?,?)") + .setDrivername(DRIVER_CLASS) + .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); } @@ -165,79 +86,84 @@ public void testIncompleteConfiguration() throws IOException { @Test(expected = IllegalArgumentException.class) public void testIncompatibleTypes() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("insert into books (id, title, author, price, qty) values (?,?,?,?,?)") + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); - Tuple5 tuple5 = new Tuple5(); tuple5.setField(4, 0); tuple5.setField("hello", 1); tuple5.setField("world", 2); tuple5.setField(0.99, 3); tuple5.setField("imthewrongtype", 4); - jdbcOutputFormat.writeRecord(tuple5); + Row row = new Row(tuple5.getArity()); + for (int i = 0; i < tuple5.getArity(); i++) { + row.setField(i, tuple5.getField(i)); + } + jdbcOutputFormat.writeRecord(row); jdbcOutputFormat.close(); } @Test - public void testJDBCOutputFormat() throws IOException { - String sourceTable = "books"; - String targetTable = "newbooks"; - String driverPath = "org.apache.derby.jdbc.EmbeddedDriver"; - String dbUrl = "jdbc:derby:memory:ebookshop"; + public void testJDBCOutputFormat() throws IOException, InstantiationException, IllegalAccessException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDBUrl(dbUrl) - .setDrivername(driverPath) - .setQuery("insert into " + targetTable + " (id, title, author, price, qty) values (?,?,?,?,?)") + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); - jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(driverPath) - .setDBUrl(dbUrl) - .setQuery("select * from " + sourceTable) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .finish(); - jdbcInputFormat.open(null); - - Tuple5 tuple = new Tuple5(); - while (!jdbcInputFormat.reachedEnd()) { - jdbcInputFormat.nextRecord(tuple); - jdbcOutputFormat.writeRecord(tuple); + for (int i = 0; i < testData.length; i++) { + Row row = new Row(testData[i].length); + for (int j = 0; j < testData[i].length; j++) { + row.setField(j, testData[i][j]); + } + jdbcOutputFormat.writeRecord(row); } jdbcOutputFormat.close(); - jdbcInputFormat.close(); - jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(driverPath) - .setDBUrl(dbUrl) - .setQuery("select * from " + targetTable) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .finish(); - jdbcInputFormat.open(null); - - int recordCount = 0; - while (!jdbcInputFormat.reachedEnd()) { - jdbcInputFormat.nextRecord(tuple); - Assert.assertEquals("Field 0 should be int", Integer.class, tuple.getField(0).getClass()); - Assert.assertEquals("Field 1 should be String", String.class, tuple.getField(1).getClass()); - Assert.assertEquals("Field 2 should be String", String.class, tuple.getField(2).getClass()); - Assert.assertEquals("Field 3 should be float", Double.class, tuple.getField(3).getClass()); - Assert.assertEquals("Field 4 should be int", Integer.class, tuple.getField(4).getClass()); - - for (int x = 0; x < 5; x++) { - Assert.assertEquals(dbData[recordCount][x], tuple.getField(x)); + try ( + Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL); + PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS); + ResultSet resultSet = statement.executeQuery() + ) { + int recordCount = 0; + while (resultSet.next()) { + Row row = new Row(tuple5.getArity()); + for (int i = 0; i < tuple5.getArity(); i++) { + row.setField(i, resultSet.getObject(i + 1)); + } + if (row.productElement(0) != null) { + Assert.assertEquals("Field 0 should be int", Integer.class, row.productElement(0).getClass()); + } + if (row.productElement(1) != null) { + Assert.assertEquals("Field 1 should be String", String.class, row.productElement(1).getClass()); + } + if (row.productElement(2) != null) { + Assert.assertEquals("Field 2 should be String", String.class, row.productElement(2).getClass()); + } + if (row.productElement(3) != null) { + Assert.assertEquals("Field 3 should be float", Double.class, row.productElement(3).getClass()); + } + if (row.productElement(4) != null) { + Assert.assertEquals("Field 4 should be int", Integer.class, row.productElement(4).getClass()); + } + + for (int x = 0; x < tuple5.getArity(); x++) { + if (JDBCTestBase.testData[recordCount][x] != null) { + Assert.assertEquals(JDBCTestBase.testData[recordCount][x], row.productElement(x)); + } + } + + recordCount++; } - - recordCount++; + Assert.assertEquals(JDBCTestBase.testData.length, recordCount); + } catch (SQLException e) { + Assert.fail("JDBC OutputFormat test failed. " + e.getMessage()); } - Assert.assertEquals(5, recordCount); - - jdbcInputFormat.close(); } } diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java new file mode 100644 index 0000000000000..1c44afe370756 --- /dev/null +++ b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java @@ -0,0 +1,183 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.api.java.io.jdbc; + +import java.io.OutputStream; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; + +/** + * Base test class for JDBC Input and Output formats + */ +public class JDBCTestBase { + + public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver"; + public static final String DB_URL = "jdbc:derby:memory:ebookshop"; + public static final String INPUT_TABLE = "books"; + public static final String OUTPUT_TABLE = "newbooks"; + public static final String SELECT_ALL_BOOKS = "select * from " + INPUT_TABLE; + public static final String SELECT_ALL_NEWBOOKS = "select * from " + OUTPUT_TABLE; + public static final String SELECT_EMPTY = "select * from books WHERE QTY < 0"; + public static final String INSERT_TEMPLATE = "insert into %s (id, title, author, price, qty) values (?,?,?,?,?)"; + public static final String SELECT_ALL_BOOKS_SPLIT_BY_ID = JDBCTestBase.SELECT_ALL_BOOKS + " WHERE id BETWEEN ? AND ?"; + public static final String SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR = JDBCTestBase.SELECT_ALL_BOOKS + " WHERE author = ?"; + + protected static Connection conn; + + public static final Object[][] testData = { + {1001, ("Java public for dummies"), ("Tan Ah Teck"), 11.11, 11}, + {1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22}, + {1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33}, + {1004, ("A Cup of Java"), ("Kumar"), 44.44, 44}, + {1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55}, + {1006, ("A Teaspoon of Java 1.4"), ("Kevin Jones"), 66.66, 66}, + {1007, ("A Teaspoon of Java 1.5"), ("Kevin Jones"), 77.77, 77}, + {1008, ("A Teaspoon of Java 1.6"), ("Kevin Jones"), 88.88, 88}, + {1009, ("A Teaspoon of Java 1.7"), ("Kevin Jones"), 99.99, 99}, + {1010, ("A Teaspoon of Java 1.8"), ("Kevin Jones"), null, 1010}}; + + public static final TypeInformation[] fieldTypes = new TypeInformation[] { + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO + }; + + public static final RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes); + + public static String getCreateQuery(String tableName) { + StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE "); + sqlQueryBuilder.append(tableName).append(" ("); + sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); + sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); + sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); + sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); + sqlQueryBuilder.append("qty INT DEFAULT NULL,"); + sqlQueryBuilder.append("PRIMARY KEY (id))"); + return sqlQueryBuilder.toString(); + } + + public static String getInsertQuery() { + StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); + for (int i = 0; i < JDBCTestBase.testData.length; i++) { + sqlQueryBuilder.append("(") + .append(JDBCTestBase.testData[i][0]).append(",'") + .append(JDBCTestBase.testData[i][1]).append("','") + .append(JDBCTestBase.testData[i][2]).append("',") + .append(JDBCTestBase.testData[i][3]).append(",") + .append(JDBCTestBase.testData[i][4]).append(")"); + if (i < JDBCTestBase.testData.length - 1) { + sqlQueryBuilder.append(","); + } + } + String insertQuery = sqlQueryBuilder.toString(); + return insertQuery; + } + + public static final OutputStream DEV_NULL = new OutputStream() { + @Override + public void write(int b) { + } + }; + + public static void prepareTestDb() throws Exception { + System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL"); + Class.forName(DRIVER_CLASS); + Connection conn = DriverManager.getConnection(DB_URL + ";create=true"); + + //create input table + Statement stat = conn.createStatement(); + stat.executeUpdate(getCreateQuery(INPUT_TABLE)); + stat.close(); + + //create output table + stat = conn.createStatement(); + stat.executeUpdate(getCreateQuery(OUTPUT_TABLE)); + stat.close(); + + //prepare input data + stat = conn.createStatement(); + stat.execute(JDBCTestBase.getInsertQuery()); + stat.close(); + + conn.close(); + } + + @BeforeClass + public static void setUpClass() throws SQLException { + try { + System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL"); + prepareDerbyDatabase(); + } catch (ClassNotFoundException e) { + e.printStackTrace(); + Assert.fail(); + } + } + + private static void prepareDerbyDatabase() throws ClassNotFoundException, SQLException { + Class.forName(DRIVER_CLASS); + conn = DriverManager.getConnection(DB_URL + ";create=true"); + createTable(INPUT_TABLE); + createTable(OUTPUT_TABLE); + insertDataIntoInputTable(); + conn.close(); + } + + private static void createTable(String tableName) throws SQLException { + Statement stat = conn.createStatement(); + stat.executeUpdate(getCreateQuery(tableName)); + stat.close(); + } + + private static void insertDataIntoInputTable() throws SQLException { + Statement stat = conn.createStatement(); + stat.execute(JDBCTestBase.getInsertQuery()); + stat.close(); + } + + @AfterClass + public static void tearDownClass() { + cleanUpDerbyDatabases(); + } + + private static void cleanUpDerbyDatabases() { + try { + Class.forName(DRIVER_CLASS); + conn = DriverManager.getConnection(DB_URL + ";create=true"); + Statement stat = conn.createStatement(); + stat.executeUpdate("DROP TABLE "+INPUT_TABLE); + stat.executeUpdate("DROP TABLE "+OUTPUT_TABLE); + stat.close(); + conn.close(); + } catch (Exception e) { + e.printStackTrace(); + Assert.fail(); + } + } + +} diff --git a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/example/JDBCExample.java b/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/example/JDBCExample.java deleted file mode 100644 index 840a31411ddf4..0000000000000 --- a/flink-batch-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/example/JDBCExample.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.java.io.jdbc.example; - -import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.DOUBLE_TYPE_INFO; -import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO; -import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.Statement; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.io.jdbc.JDBCInputFormat; -import org.apache.flink.api.java.io.jdbc.JDBCOutputFormat; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; - -public class JDBCExample { - - public static void main(String[] args) throws Exception { - prepareTestDb(); - - ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment(); - DataSet source - = environment.createInput(JDBCInputFormat.buildJDBCInputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("select * from books") - .finish(), - new TupleTypeInfo(Tuple5.class, INT_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO, DOUBLE_TYPE_INFO, INT_TYPE_INFO) - ); - - source.output(JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername("org.apache.derby.jdbc.EmbeddedDriver") - .setDBUrl("jdbc:derby:memory:ebookshop") - .setQuery("insert into newbooks (id,title,author,price,qty) values (?,?,?,?,?)") - .finish()); - environment.execute(); - } - - private static void prepareTestDb() throws Exception { - System.setProperty("derby.stream.error.field", "org.apache.flink.api.java.io.jdbc.DerbyUtil.DEV_NULL"); - String dbURL = "jdbc:derby:memory:ebookshop;create=true"; - Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - Connection conn = DriverManager.getConnection(dbURL); - - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE books ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - Statement stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - - sqlQueryBuilder = new StringBuilder("CREATE TABLE newbooks ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - - stat = conn.createStatement(); - stat.executeUpdate(sqlQueryBuilder.toString()); - stat.close(); - - sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - sqlQueryBuilder.append("(1001, 'Java for dummies', 'Tan Ah Teck', 11.11, 11),"); - sqlQueryBuilder.append("(1002, 'More Java for dummies', 'Tan Ah Teck', 22.22, 22),"); - sqlQueryBuilder.append("(1003, 'More Java for more dummies', 'Mohammad Ali', 33.33, 33),"); - sqlQueryBuilder.append("(1004, 'A Cup of Java', 'Kumar', 44.44, 44),"); - sqlQueryBuilder.append("(1005, 'A Teaspoon of Java', 'Kevin Jones', 55.55, 55)"); - - stat = conn.createStatement(); - stat.execute(sqlQueryBuilder.toString()); - stat.close(); - - conn.close(); - } -} From 0109a8af825fd3212d4859cd37258b989a4e2c76 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 19 May 2016 11:25:22 +0200 Subject: [PATCH 34/70] [hotfix] [tableAPI] Adapt tests to changed null-value support default --- .../flink/api/scala/batch/utils/TableProgramsTestBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala index 37aaa5298c242..41e71d37a8252 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala @@ -38,7 +38,7 @@ class TableProgramsTestBase( val conf = new TableConfig tableConfigMode match { case NULL => - conf.setNullCheck(true) + conf.setNullCheck(false) case EFFICIENT => conf.setEfficientTypeUsage(true) case _ => // keep default From 816a6004ac3923aa57f6844150a74161e3407535 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 19 May 2016 01:47:29 +0200 Subject: [PATCH 35/70] [FLINK-3934] [tableAPI] Check for equi-join predicates before translation. --- .../plan/nodes/dataset/DataSetJoin.scala | 67 ++----------------- .../plan/rules/dataSet/DataSetJoinRule.scala | 58 +++++++++------- .../api/scala/batch/sql/JoinITCase.scala | 2 +- .../api/scala/batch/table/JoinITCase.scala | 7 +- 4 files changed, 44 insertions(+), 90 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala index 4f24f4e18a6ec..cdf7461084cba 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala @@ -23,7 +23,6 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.JoinInfo import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelWriter, BiRel, RelNode} -import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.util.mapping.IntPair import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint import org.apache.flink.api.common.typeinfo.TypeInformation @@ -31,12 +30,11 @@ import org.apache.flink.api.java.DataSet import org.apache.flink.api.java.operators.join.JoinType import org.apache.flink.api.table.codegen.CodeGenerator import org.apache.flink.api.table.runtime.FlatJoinRunner -import org.apache.flink.api.table.typeutils.TypeConverter +import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType import org.apache.flink.api.table.{BatchTableEnvironment, TableException} import org.apache.flink.api.common.functions.FlatJoinFunction -import TypeConverter.determineReturnType import scala.collection.mutable.ArrayBuffer -import org.apache.calcite.rex.{RexInputRef, RexCall, RexNode} +import org.apache.calcite.rex.RexNode import scala.collection.JavaConverters._ import scala.collection.JavaConversions._ @@ -60,8 +58,6 @@ class DataSetJoin( extends BiRel(cluster, traitSet, left, right) with DataSetRel { - val translatable = canBeTranslated - override def deriveRowType() = rowType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { @@ -92,19 +88,12 @@ class DataSetJoin( override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { - if (!translatable) { - // join cannot be translated. Make huge costs - planner.getCostFactory.makeHugeCost() - } else { - // join can be translated. Compute cost estimate - val children = this.getInputs - children.foldLeft(planner.getCostFactory.makeCost(0, 0, 0)) { (cost, child) => - val rowCnt = metadata.getRowCount(child) - val rowSize = this.estimateRowSize(child.getRowType) - cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)) - } + val children = this.getInputs + children.foldLeft(planner.getCostFactory.makeCost(0, 0, 0)) { (cost, child) => + val rowCnt = metadata.getRowCount(child) + val rowSize = this.estimateRowSize(child.getRowType) + cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)) } - } override def translateToPlan( @@ -204,48 +193,6 @@ class DataSetJoin( .`with`(joinFun).name(joinOpName).asInstanceOf[DataSet[Any]] } - private def canBeTranslated: Boolean = { - - val equiCondition = - joinInfo.getEquiCondition(left, right, cluster.getRexBuilder) - - // joins require at least one equi-condition - if (equiCondition.isAlwaysTrue) { - false - } - else { - // check that all equality predicates refer to field refs only (not computed expressions) - // Note: Calcite treats equality predicates on expressions as non-equi predicates - joinCondition match { - - // conjunction of join predicates - case c: RexCall if c.getOperator.equals(SqlStdOperatorTable.AND) => - - c.getOperands.asScala - // look at equality predicates only - .filter { o => - o.isInstanceOf[RexCall] && - o.asInstanceOf[RexCall].getOperator.equals(SqlStdOperatorTable.EQUALS) - } - // check that both children are field references - .map { o => - o.asInstanceOf[RexCall].getOperands.get(0).isInstanceOf[RexInputRef] && - o.asInstanceOf[RexCall].getOperands.get(1).isInstanceOf[RexInputRef] - } - // any equality predicate that does not refer to a field reference? - .reduce( (a, b) => a && b) - - // single equi-join predicate - case c: RexCall if c.getOperator.equals(SqlStdOperatorTable.EQUALS) => - c.getOperands.get(0).isInstanceOf[RexInputRef] && - c.getOperands.get(1).isInstanceOf[RexInputRef] - case _ => - false - } - } - - } - private def joinSelectionToString: String = { rowType.getFieldNames.asScala.toList.mkString(", ") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala index 55100d20af752..f3bd402fd01ac 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala @@ -25,6 +25,7 @@ import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rel.logical.LogicalJoin import org.apache.flink.api.java.operators.join.JoinType import org.apache.flink.api.table.plan.nodes.dataset.{DataSetJoin, DataSetConvention} + import scala.collection.JavaConversions._ class DataSetJoinRule @@ -32,40 +33,45 @@ class DataSetJoinRule classOf[LogicalJoin], Convention.NONE, DataSetConvention.INSTANCE, - "DataSetJoinRule") - { + "DataSetJoinRule") { - /** - * Only translate INNER joins for now - */ override def matches(call: RelOptRuleCall): Boolean = { val join: LogicalJoin = call.rel(0).asInstanceOf[LogicalJoin] - join.getJoinType.equals(JoinRelType.INNER) + + val joinInfo = join.analyzeCondition + + // joins require an equi-condition or a conjunctive predicate with at least one equi-condition + val hasValidCondition = !joinInfo.pairs().isEmpty + // only inner joins are supported at the moment + val isInnerJoin = join.getJoinType.equals(JoinRelType.INNER) + + // check that condition is valid and inner join + hasValidCondition && isInnerJoin } - def convert(rel: RelNode): RelNode = { + override def convert(rel: RelNode): RelNode = { - val join: LogicalJoin = rel.asInstanceOf[LogicalJoin] - val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE) - val convLeft: RelNode = RelOptRule.convert(join.getInput(0), DataSetConvention.INSTANCE) - val convRight: RelNode = RelOptRule.convert(join.getInput(1), DataSetConvention.INSTANCE) - val joinInfo = join.analyzeCondition + val join: LogicalJoin = rel.asInstanceOf[LogicalJoin] + val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE) + val convLeft: RelNode = RelOptRule.convert(join.getInput(0), DataSetConvention.INSTANCE) + val convRight: RelNode = RelOptRule.convert(join.getInput(1), DataSetConvention.INSTANCE) + val joinInfo = join.analyzeCondition - new DataSetJoin( - rel.getCluster, - traitSet, - convLeft, - convRight, - rel.getRowType, - join.getCondition, - join.getRowType, - joinInfo, - joinInfo.pairs.toList, - JoinType.INNER, - null, - description) - } + new DataSetJoin( + rel.getCluster, + traitSet, + convLeft, + convRight, + rel.getRowType, + join.getCondition, + join.getRowType, + joinInfo, + joinInfo.pairs.toList, + JoinType.INNER, + null, + description) } +} object DataSetJoinRule { val INSTANCE: RelOptRule = new DataSetJoinRule diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala index 40d75467f72eb..d388c33d9d8d5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala @@ -190,7 +190,7 @@ class JoinITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[PlanGenException]) def testJoinNoEqualityPredicate(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala index 1a9be93c330e5..ae76acecbf773 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala @@ -21,8 +21,9 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets +import org.apache.flink.api.table.plan.PlanGenException +import org.apache.flink.api.table.{ValidationException, Row, TableEnvironment} import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} import org.junit._ @@ -139,7 +140,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .select('c, 'g) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[PlanGenException]) def testNoEqualityJoinPredicate1(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -153,7 +154,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .select('c, 'g).collect() } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[PlanGenException]) def testNoEqualityJoinPredicate2(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) From f6bcd9cbd7c531c3022ae2745b69260ddca05628 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 19 May 2016 00:09:00 +0200 Subject: [PATCH 36/70] [FLINK-3935] [optimizer] Fix check of key fields and ordering in PartitionNode. --- .../main/java/org/apache/flink/optimizer/dag/PartitionNode.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java index 63ae1add054a2..2d3c6463f598b 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java @@ -98,7 +98,7 @@ public PartitionDescriptor(PartitionMethod pMethod, FieldSet pKeys, Ordering ord super(pKeys); Preconditions.checkArgument(pMethod != PartitionMethod.RANGE - || pKeys.toFieldList().isExactMatch(ordering.getInvolvedIndexes()), + || pKeys.equals(new FieldSet(ordering.getFieldPositions())), "Partition keys must match the given ordering."); this.pMethod = pMethod; From 7f54dfbc88ab5c54ea5914a6b526394d591447be Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 19 May 2016 14:58:15 +0200 Subject: [PATCH 37/70] [hotfix] [docs] Remove misleading sentence from DataSet.union() description. --- docs/apis/batch/index.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/apis/batch/index.md b/docs/apis/batch/index.md index 4c0aea8472b01..2f7013f866679 100644 --- a/docs/apis/batch/index.md +++ b/docs/apis/batch/index.md @@ -354,8 +354,7 @@ DataSet> result = data1.cross(data2); Union -

Produces the union of two data sets. This operation happens implicitly if more than one - data set is used for a specific function input.

+

Produces the union of two data sets.

{% highlight java %} DataSet data1 = // [...] DataSet data2 = // [...] From 08eb5570f4699524c18ad548898ea92cd5e17a87 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Thu, 19 May 2016 13:30:09 -0700 Subject: [PATCH 38/70] [script] Simple fix of typo from JobManager to TaskManager in taskmanager.sh file Very simple fix for typo on comment in the task manager.sh file. From `JobManager` to `TaskManager`. Author: Henry Saputra Closes #2001 from hsaputra/simple_text_fix_taskmanager and squashes the following commits: 861cbb4 [Henry Saputra] Simple fix of typo from JobManager to TaskManager in taskmanager.sh file. --- flink-dist/src/main/flink-bin/bin/taskmanager.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-dist/src/main/flink-bin/bin/taskmanager.sh b/flink-dist/src/main/flink-bin/bin/taskmanager.sh index 87860e80d0a59..14adcafef0565 100755 --- a/flink-dist/src/main/flink-bin/bin/taskmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/taskmanager.sh @@ -17,7 +17,7 @@ # limitations under the License. ################################################################################ -# Start/stop a Flink JobManager. +# Start/stop a Flink TaskManager. USAGE="Usage: taskmanager.sh (start|stop|stop-all)" STARTSTOP=$1 From 396691ff8a1456bf7e770f1b3db4b0542fd0c663 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 19 May 2016 12:37:05 +0200 Subject: [PATCH 39/70] [FLINK-3933] [streaming API] Add AbstractDeserializationSchema that handles produced type extraction. The AbstractDeserializationSchema creates the produced type information automatically from the generic parameters. This closes #2010 --- docs/apis/streaming/connectors/kafka.md | 19 ++- .../api/java/typeutils/TypeExtractor.java | 4 +- .../AbstractDeserializationSchema.java | 68 ++++++++++ .../serialization/DeserializationSchema.java | 3 + .../serialization/SimpleStringSchema.java | 3 +- .../AbstractDeserializationSchemaTest.java | 116 ++++++++++++++++++ 6 files changed, 203 insertions(+), 10 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java diff --git a/docs/apis/streaming/connectors/kafka.md b/docs/apis/streaming/connectors/kafka.md index da3f86ca78dc2..45ec6adb501c7 100644 --- a/docs/apis/streaming/connectors/kafka.md +++ b/docs/apis/streaming/connectors/kafka.md @@ -87,13 +87,13 @@ Then, import the connector in your maven project: Note that the streaming connectors are currently not part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution). -#### Installing Apache Kafka +### Installing Apache Kafka * Follow the instructions from [Kafka's quickstart](https://kafka.apache.org/documentation.html#quickstart) to download the code and launch a server (launching a Zookeeper and a Kafka server is required every time before starting the application). * On 32 bit computers [this](http://stackoverflow.com/questions/22325364/unrecognized-vm-option-usecompressedoops-when-running-kafka-from-my-ubuntu-in) problem may occur. * If the Kafka and Zookeeper servers are running on a remote machine, then the `advertised.host.name` setting in the `config/server.properties` file must be set to the machine's IP address. -#### Kafka Consumer +### Kafka Consumer Flink's Kafka consumer is called `FlinkKafkaConsumer08` (or `09` for Kafka 0.9.0.x versions). It provides access to one or more Kafka topics. @@ -142,18 +142,25 @@ for querying the list of topics and partitions. For this to work, the consumer needs to be able to access the consumers from the machine submitting the job to the Flink cluster. If you experience any issues with the Kafka consumer on the client side, the client log might contain information about failed requests, etc. -##### The `DeserializationSchema` +#### The `DeserializationSchema` -The `FlinkKafkaConsumer08` needs to know how to turn the data in Kafka into Java objects. The +The Flink Kafka Consumer needs to know how to turn the binary data in Kafka into Java/Scala objects. The `DeserializationSchema` allows users to specify such a schema. The `T deserialize(byte[] message)` method gets called for each Kafka message, passing the value from Kafka. +It is usually helpful to start from the `AbstractDeserializationSchema`, which takes care of describing the +produced Java/Scala type to Flink's type system. Users that implement a vanilla `DeserializationSchema` need +to implement the `getProducedType(...)` method themselves. + For accessing both the key and value of the Kafka message, the `KeyedDeserializationSchema` has the following deserialize method ` T deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset)`. For convenience, Flink provides the following schemas: + 1. `TypeInformationSerializationSchema` (and `TypeInformationKeyValueSerializationSchema`) which creates - a schema based on a Flink `TypeInformation`. + a schema based on a Flink's `TypeInformation`. This is useful if the data is both written and read by Flink. + This schema is a performant Flink-specific alternative to other generic serialization approaches. + 2. `JsonDeserializationSchema` (and `JSONKeyValueDeserializationSchema`) which turns the serialized JSON into an ObjectNode object, from which fields can be accessed using objectNode.get("field").as(Int/String/...)(). The KeyValue objectNode contains a "key" and "value" field which contain all fields, as well as @@ -191,7 +198,7 @@ Flink on YARN supports automatic restart of lost YARN containers. If checkpointing is not enabled, the Kafka consumer will periodically commit the offsets to Zookeeper. -#### Kafka Producer +### Kafka Producer The `FlinkKafkaProducer08` writes data to a Kafka topic. The producer can specify a custom partitioner that assigns records to partitions. diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index f2b9fd23981cf..45420a26787d5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -272,7 +272,7 @@ public static TypeInformation getInputFormatTypes(InputFormat in } return new TypeExtractor().privateCreateTypeInfo(InputFormat.class, inputFormatInterface.getClass(), 0, null, null); } - + // -------------------------------------------------------------------------------------------- // Generic extraction methods // -------------------------------------------------------------------------------------------- @@ -596,7 +596,7 @@ private TypeInformation createTypeInfoWithTypeHierarchy(Arr } if(curT == Tuple0.class) { - return new TupleTypeInfo(Tuple0.class, new TypeInformation[0]); + return new TupleTypeInfo(Tuple0.class); } // check if immediate child of Tuple has generics diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java new file mode 100644 index 0000000000000..77e76e5b5c96d --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.util.serialization; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.TypeExtractor; + +import java.io.IOException; + +/** + * The deserialization schema describes how to turn the byte messages delivered by certain + * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are + * processed by Flink. + * + *

This base variant of the deserialization schema produces the type information + * automatically by extracting it from the generic class arguments. + * + * @param The type created by the deserialization schema. + */ +public abstract class AbstractDeserializationSchema implements DeserializationSchema { + + private static final long serialVersionUID = 1L; + + /** + * De-serializes the byte message. + * + * @param message The message, as a byte array. + * @return The de-serialized message as an object. + */ + @Override + public abstract T deserialize(byte[] message) throws IOException; + + /** + * Method to decide whether the element signals the end of the stream. If + * true is returned the element won't be emitted. + * + *

This default implementation returns always false, meaning the stream is interpreted + * to be unbounded. + * + * @param nextElement The element to test for the end-of-stream signal. + * @return True, if the element signals end of stream, false otherwise. + */ + @Override + public boolean isEndOfStream(T nextElement) { + return false; + } + + @Override + public TypeInformation getProducedType() { + return TypeExtractor.createTypeInfo(AbstractDeserializationSchema.class, getClass(), 0, null, null); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java index b376d69d20531..2e27ba61b5b7f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java @@ -28,6 +28,9 @@ * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are * processed by Flink. * + *

Note: In most cases, one should start from {@link AbstractDeserializationSchema}, which + * takes care of producing the return type information automatically. + * * @param The type created by the deserialization schema. */ @Public diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java index a051d324fe49a..2de4c013dcc4f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java @@ -25,8 +25,7 @@ * Very simple serialization schema for strings. */ @PublicEvolving -public class SimpleStringSchema implements DeserializationSchema, - SerializationSchema { +public class SimpleStringSchema implements DeserializationSchema, SerializationSchema { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java new file mode 100644 index 0000000000000..220c1cd23391b --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.util; + +import org.apache.flink.api.common.functions.InvalidTypesException; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.util.serialization.AbstractDeserializationSchema; + +import org.codehaus.jackson.map.util.JSONPObject; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.*; + +@SuppressWarnings("serial") +public class AbstractDeserializationSchemaTest { + + @Test + public void testTypeExtractionTuple() { + TypeInformation> type = new TupleSchema().getProducedType(); + TypeInformation> expected = TypeInformation.of(new TypeHint>(){}); + assertEquals(expected, type); + } + + @Test + public void testTypeExtractionTupleAnonymous() { + TypeInformation> type = new AbstractDeserializationSchema>() { + @Override + public Tuple2 deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException(); + } + }.getProducedType(); + + TypeInformation> expected = TypeInformation.of(new TypeHint>(){}); + assertEquals(expected, type); + } + + @Test + public void testTypeExtractionGeneric() { + TypeInformation type = new JsonSchema().getProducedType(); + TypeInformation expected = TypeInformation.of(new TypeHint(){}); + assertEquals(expected, type); + } + + @Test + public void testTypeExtractionGenericAnonymous() { + TypeInformation type = new AbstractDeserializationSchema() { + @Override + public JSONPObject deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException(); + } + }.getProducedType(); + + TypeInformation expected = TypeInformation.of(new TypeHint(){}); + assertEquals(expected, type); + } + + @Test + public void testTypeExtractionRawException() { + try { + new RawSchema().getProducedType(); + fail(); + } catch (InvalidTypesException e) { + // expected + } + } + + // ------------------------------------------------------------------------ + // Test types + // ------------------------------------------------------------------------ + + private static class TupleSchema extends AbstractDeserializationSchema> { + + @Override + public Tuple2 deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException(); + } + } + + private static class JsonSchema extends AbstractDeserializationSchema { + + @Override + public JSONPObject deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException(); + } + } + + @SuppressWarnings("rawtypes") + private static class RawSchema extends AbstractDeserializationSchema { + + @Override + public Object deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException(); + } + } +} From 470094f976882b4ca0c40a6a7f80a9e6e8ff6f29 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 3 May 2016 15:25:33 +0200 Subject: [PATCH 40/70] [FLINK-3892] ConnectionUtils may die with NullPointerException This fixes a NullPointerException that would occur if the host name could not be resolved. This closes #2008 --- .../java/org/apache/flink/runtime/net/ConnectionUtils.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java index a03183deda252..77324fa3ba6ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/ConnectionUtils.java @@ -240,7 +240,11 @@ private static InetAddress findAddressUsingStrategy(AddressDetectionState strate } } - final byte[] targetAddressBytes = targetAddress.getAddress().getAddress(); + final InetAddress address = targetAddress.getAddress(); + if (address == null) { + return null; + } + final byte[] targetAddressBytes = address.getAddress(); // for each network interface Enumeration e = NetworkInterface.getNetworkInterfaces(); From d0eee64dbdd0181112ace561934beadf7fe37ffd Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 10 May 2016 16:45:48 +0200 Subject: [PATCH 41/70] [FLINK-3893] improve LeaderChangeStateCleanupTest stability - increase timeout for task managers to be registered - don't fail if leader retrieval service has not been started yet This closes #2009 --- .../leaderelection/LeaderChangeStateCleanupTest.java | 12 ++++++------ .../TestingLeaderRetrievalService.java | 2 -- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java index 2eacdeec1b1e3..89f462c5d8d7b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java @@ -96,7 +96,7 @@ public void testStateCleanupAfterNewLeaderElectionAndListenerNotification() thro // notify all listeners cluster.notifyRetrievalListeners(0, leaderSessionID1); - cluster.waitForTaskManagersToBeRegistered(); + cluster.waitForTaskManagersToBeRegistered(timeout); // submit blocking job so that it is not finished when we cancel it cluster.submitJobDetached(job); @@ -116,7 +116,7 @@ public void testStateCleanupAfterNewLeaderElectionAndListenerNotification() thro Await.ready(jobRemoval, timeout); - cluster.waitForTaskManagersToBeRegistered(); + cluster.waitForTaskManagersToBeRegistered(timeout); ActorGateway jm2 = cluster.getLeaderGateway(timeout); @@ -145,7 +145,7 @@ public void testStateCleanupAfterNewLeaderElection() throws Exception { cluster.grantLeadership(0, leaderSessionID); cluster.notifyRetrievalListeners(0, leaderSessionID); - cluster.waitForTaskManagersToBeRegistered(); + cluster.waitForTaskManagersToBeRegistered(timeout); // submit blocking job so that we can test job clean up cluster.submitJobDetached(job); @@ -178,7 +178,7 @@ public void testStateCleanupAfterListenerNotification() throws Exception { cluster.grantLeadership(0, leaderSessionID); cluster.notifyRetrievalListeners(0, leaderSessionID); - cluster.waitForTaskManagersToBeRegistered(); + cluster.waitForTaskManagersToBeRegistered(timeout); // submit blocking job cluster.submitJobDetached(job); @@ -212,7 +212,7 @@ public void testReelectionOfSameJobManager() throws Exception { cluster.grantLeadership(0, leaderSessionID); cluster.notifyRetrievalListeners(0, leaderSessionID); - cluster.waitForTaskManagersToBeRegistered(); + cluster.waitForTaskManagersToBeRegistered(timeout); // submit blocking job cluster.submitJobDetached(job); @@ -242,7 +242,7 @@ public void testReelectionOfSameJobManager() throws Exception { // notify the TMs about the new (old) leader cluster.notifyRetrievalListeners(0, newLeaderSessionID); - cluster.waitForTaskManagersToBeRegistered(); + cluster.waitForTaskManagersToBeRegistered(timeout); ActorGateway leaderGateway = cluster.getLeaderGateway(timeout); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java index e2fc54028f300..c5eb155f6ba1b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java @@ -61,8 +61,6 @@ public void stop() throws Exception { public void notifyListener(String address, UUID leaderSessionID) { if (listener != null) { listener.notifyLeaderAddress(address, leaderSessionID); - } else { - throw new IllegalStateException("The retrieval service has not been started properly."); } } } From 23ea8119a9fcd2c91c7d7d1e814baa5d34feca0c Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 18 May 2016 14:34:06 +0200 Subject: [PATCH 42/70] [FLINK-3909] replace Maven Failsafe with Surefire Failures during execution of the integration tests with the Maven Failsafe plugin were silently ignored on Travis with Maven version 3.2.5. The problem is that failures are not passed on correctly from the 'integration-test' phase (where failures are recorded and tolerated) to the 'verify' phase. The cause of the error is most likely SUREFIRE-1127. An exception in the 'integration-test' is sometimes not flushed back to disk where it is evaluated in the 'verify' phase. Bumping the version of the Failsafe plugin from 2.18.1 to 2.19.1 fixes the issue but introduces classpath related problems. We don't have to rely on Failsafe for running integration tests. Surefire offers the same feature set we use from the Failsafe plugin. - replace Failsafe with Surefire - update to latest Surefire version - get rid of duplicate property declaration This closes #2003 --- flink-java8/pom.xml | 8 --- flink-runtime/pom.xml | 13 ----- .../flink-connector-elasticsearch/pom.xml | 7 --- .../flink-connector-kafka-0.8/pom.xml | 2 +- .../flink-connector-kafka-0.9/pom.xml | 9 --- .../flink-connector-kafka-base/pom.xml | 9 --- .../flink-connector-nifi/pom.xml | 7 --- flink-tests/pom.xml | 16 +----- flink-yarn-tests/pom.xml | 11 +--- pom.xml | 56 ++++++++++--------- 10 files changed, 35 insertions(+), 103 deletions(-) diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml index 5ec7ea904642d..c8adcb81406e8 100644 --- a/flink-java8/pom.xml +++ b/flink-java8/pom.xml @@ -101,14 +101,6 @@ under the License. - - maven-failsafe-plugin - - - WARN - - - diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 7fa967ec53e2c..13c4fa51d0c86 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -313,19 +313,6 @@ under the License. - - org.apache.maven.plugins - maven-failsafe-plugin - - false - - **/TestData.java - - - WARN - - - diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml b/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml index e7d95b591fca9..7ddf7e3a42a9b 100644 --- a/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml +++ b/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml @@ -95,13 +95,6 @@ under the License. 3 - - org.apache.maven.plugins - maven-failsafe-plugin - - 3 - - diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml index f62be43fd5cc7..b2701c137288d 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/pom.xml @@ -144,7 +144,7 @@ under the License. org.apache.maven.plugins - maven-failsafe-plugin + maven-surefire-plugin 1 diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml index f0bd98ef899ce..e45a1d01a0c28 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml @@ -155,15 +155,6 @@ under the License. -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit - - org.apache.maven.plugins - maven-failsafe-plugin - - - 1 - -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit - - diff --git a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml index ec1514fcb7bfe..e430ff4eee9e8 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml @@ -151,15 +151,6 @@ under the License. - - - org.apache.maven.plugins - maven-failsafe-plugin - - - 1 - - diff --git a/flink-streaming-connectors/flink-connector-nifi/pom.xml b/flink-streaming-connectors/flink-connector-nifi/pom.xml index 82fb5c1ebb3fc..d93bce79f029b 100644 --- a/flink-streaming-connectors/flink-connector-nifi/pom.xml +++ b/flink-streaming-connectors/flink-connector-nifi/pom.xml @@ -83,13 +83,6 @@ under the License. 3 - - org.apache.maven.plugins - maven-failsafe-plugin - - 3 - - diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 0459039d70561..83f004b69bca5 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -333,21 +333,7 @@ under the License. false - - - maven-failsafe-plugin - - - WARN - - - org.apache.curator:curator-recipes - org.apache.curator:curator-client - org.apache.curator:curator-framework - - - - + maven-assembly-plugin 2.4 diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 255eeee679b69..bf5b5ce23c52b 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -93,21 +93,14 @@ under the License. --> org.apache.maven.plugins - maven-failsafe-plugin + maven-surefire-plugin 1 ../ - - org.apache.maven.plugins - maven-surefire-plugin - - ../ - - - + org.apache.maven.plugins maven-shade-plugin diff --git a/pom.xml b/pom.xml index 5d3814a0d5e3e..1ba237f596e9b 100644 --- a/pom.xml +++ b/pom.xml @@ -85,7 +85,7 @@ under the License. error 1.2.1 2.3.0 - 1.5C @@ -906,40 +906,46 @@ under the License. -Xlint:all - - maven-failsafe-plugin - 2.18.1 - - - - integration-test - verify - - - - - ${flink.forkCount} - false - - - 0${surefire.forkNumber} - - -Xms256m -Xmx800m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit - - org.apache.maven.plugins maven-surefire-plugin - 2.18.1 + 2.19.1 ${flink.forkCount} ${flink.reuseForks} 0${surefire.forkNumber} + ${log4j.configuration} - -Xms256m -Xmx800m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + -Xms256m -Xmx800m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + + + unit-tests + test + + test + + + + **/*ITCase.* + + + + + integration-tests + integration-test + + test + + + + **/*ITCase.* + + false + + + org.apache.maven.plugins From b8faf3780757bbc3424db8141a62ee32c50ac7dc Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 19 May 2016 17:59:23 +0200 Subject: [PATCH 43/70] [FLINK-3938] re-enable Yarn tests As of 70978f560fa5cab6d84ec27d58faa2627babd362, the Yarn tests were not executed anymore. They were moved to the test directory but there was still a Maven configuration in place to change the test directory location. This closes #2012 --- flink-yarn-tests/pom.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index bf5b5ce23c52b..e8e1ec913917c 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -85,8 +85,6 @@ under the License. - src/main/java - + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + org.apache.avro diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index a23964e572ef7..d623707187544 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -40,6 +40,7 @@ import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.metrics.MetricGroup; /** * A RuntimeContext contains information about the context in which functions are executed. Each parallel instance @@ -59,6 +60,13 @@ public interface RuntimeContext { */ String getTaskName(); + /** + * Returns the metric group for this parallel subtask. + * + * @return The metric group for this parallel subtask. + */ + MetricGroup getMetricGroup(); + /** * Gets the parallelism with which the parallel task runs. * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java index 9b139c61e5569..265f925681cb6 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java @@ -44,6 +44,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -62,17 +63,21 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext { private final Map> accumulators; private final DistributedCache distributedCache; + + private final MetricGroup metrics; public AbstractRuntimeUDFContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, Map> accumulators, - Map> cpTasks) { + Map> cpTasks, + MetricGroup metrics) { this.taskInfo = checkNotNull(taskInfo); this.userCodeClassLoader = userCodeClassLoader; this.executionConfig = executionConfig; this.distributedCache = new DistributedCache(checkNotNull(cpTasks)); this.accumulators = checkNotNull(accumulators); + this.metrics = metrics; } @Override @@ -94,6 +99,11 @@ public int getNumberOfParallelSubtasks() { public int getIndexOfThisSubtask() { return taskInfo.getIndexOfThisSubtask(); } + + @Override + public MetricGroup getMetricGroup() { + return metrics; + } @Override public int getAttemptNumber() { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java index 2337afaa5f5be..6571d0d44686e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; /** * A standalone implementation of the {@link RuntimeContext}, created by runtime UDF operators. @@ -42,8 +43,9 @@ public class RuntimeUDFContext extends AbstractRuntimeUDFContext { private final HashMap> uninitializedBroadcastVars = new HashMap>(); public RuntimeUDFContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulators) { - super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks); + Map> cpTasks, Map> accumulators, + MetricGroup metrics) { + super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks, metrics); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java index 44042c432db2d..7e5269e84fe87 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java @@ -36,6 +36,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; @@ -58,9 +59,15 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.metrics.groups.JobMetricGroup; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.types.Value; +import org.apache.flink.util.AbstractID; import org.apache.flink.util.Visitor; /** @@ -86,6 +93,8 @@ public class CollectionExecutor { private final ExecutionConfig executionConfig; private int iterationSuperstep; + + private JobMetricGroup jobMetricGroup; // -------------------------------------------------------------------------------------------- @@ -106,6 +115,14 @@ public CollectionExecutor(ExecutionConfig executionConfig) { public JobExecutionResult execute(Plan program) throws Exception { long startTime = System.currentTimeMillis(); + + JobID jobID = program.getJobId(); + if (jobID == null) { + jobID = new JobID(); + } + this.jobMetricGroup = + new TaskManagerMetricGroup(new MetricRegistry(new Configuration()), "localhost", new AbstractID().toString()) + .addJob(jobID, program.getJobName()); initCache(program.getCachedFiles()); Collection> sinks = program.getDataSinks(); for (Operator sink : sinks) { @@ -184,9 +201,12 @@ private void executeDataSink(GenericDataSinkBase sink, int superStep) th // build the runtime context and compute broadcast variables, if necessary TaskInfo taskInfo = new TaskInfo(typedSink.getName(), 0, 1, 0); RuntimeUDFContext ctx; + + MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, typedSink.getName()); + if (RichOutputFormat.class.isAssignableFrom(typedSink.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); } else { ctx = null; } @@ -200,10 +220,13 @@ private List executeDataSource(GenericDataSourceBase source, in GenericDataSourceBase typedSource = (GenericDataSourceBase) source; // build the runtime context and compute broadcast variables, if necessary TaskInfo taskInfo = new TaskInfo(typedSource.getName(), 0, 1, 0); + RuntimeUDFContext ctx; + + MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, source.getName()); if (RichInputFormat.class.isAssignableFrom(typedSource.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); } else { ctx = null; } @@ -225,9 +248,11 @@ private List executeUnaryOperator(SingleInputOperator op // build the runtime context and compute broadcast variables, if necessary TaskInfo taskInfo = new TaskInfo(typedOp.getName(), 0, 1, 0); RuntimeUDFContext ctx; + + MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, typedOp.getName()); if (RichFunction.class.isAssignableFrom(typedOp.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); for (Map.Entry> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -265,9 +290,11 @@ private List executeBinaryOperator(DualInputOperator> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -523,8 +550,9 @@ else if (op instanceof GenericDataSourceBase) { private class IterationRuntimeUDFContext extends RuntimeUDFContext implements IterationRuntimeContext { public IterationRuntimeUDFContext(TaskInfo taskInfo, ClassLoader classloader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulators) { - super(taskInfo, classloader, executionConfig, cpTasks, accumulators); + Map> cpTasks, Map> accumulators, + MetricGroup metrics) { + super(taskInfo, classloader, executionConfig, cpTasks, accumulators, metrics); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/metrics/Counter.java b/flink-core/src/main/java/org/apache/flink/metrics/Counter.java new file mode 100644 index 0000000000000..b18da4fe2672e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/Counter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A Counter is a {@link org.apache.flink.metrics.Metric} that measures a count. + */ +@PublicEvolving +public final class Counter implements Metric { + private long count = 0; + + /** + * Increment the current count by 1. + */ + public void inc() { + count++; + } + + /** + * Increment the current count by the given value. + * + * @param n value to increment the current count by + */ + public void inc(long n) { + count += n; + } + + /** + * Decrement the current count by 1. + */ + public void dec() { + count--; + } + + /** + * Decrement the current count by the given value. + * + * @param n value to decrement the current count by + */ + public void dec(long n) { + count -= n; + } + + /** + * Returns the current count. + * + * @return current count + */ + public long getCount() { + return count; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/Gauge.java b/flink-core/src/main/java/org/apache/flink/metrics/Gauge.java new file mode 100644 index 0000000000000..455587ad743eb --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/Gauge.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A Gauge is a {@link org.apache.flink.metrics.Metric} that calculates a specific value at a point in time. + */ +@PublicEvolving +public abstract class Gauge implements Metric { + /** + * Calculates and returns the measured value. + * + * @return calculated value + */ + public abstract T getValue(); +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/Metric.java b/flink-core/src/main/java/org/apache/flink/metrics/Metric.java new file mode 100644 index 0000000000000..11cfcc6df3831 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/Metric.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Common interface for all metrics. + */ +@PublicEvolving +public interface Metric { +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java new file mode 100644 index 0000000000000..403612958e7d5 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A MetricGroup is a named container for {@link org.apache.flink.metrics.Metric}s and + * {@link org.apache.flink.metrics.MetricGroup}s. + *

+ * Instances of this class can be used to register new metrics with Flink and to create a nested hierarchy based on the + * group names. + *

+ * A MetricGroup is uniquely identified by it's place in the hierarchy and name. + */ +@PublicEvolving +public interface MetricGroup { + + /** + * Recursively unregisters all {@link org.apache.flink.metrics.Metric}s contained in this + * {@link org.apache.flink.metrics.MetricGroup} + */ + void close(); + + // ----------------------------------------------------------------------------------------------------------------- + // Metrics + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Creates and registers a new {@link org.apache.flink.metrics.Counter} with Flink. + * + * @param name name of the counter + * @return the registered counter + */ + Counter counter(int name); + + /** + * Creates and registers a new {@link org.apache.flink.metrics.Counter} with Flink. + * + * @param name name of the counter + * @return the registered counter + */ + Counter counter(String name); + + /** + * Registers a new {@link org.apache.flink.metrics.Gauge} with Flink. + * + * @param name name of the gauge + * @param gauge gauge to register + * @param return type of the gauge + * @return the registered gauge + */ + Gauge gauge(int name, Gauge gauge); + + /** + * Registers a new {@link org.apache.flink.metrics.Gauge} with Flink. + * + * @param name name of the gauge + * @param gauge gauge to register + * @param return type of the gauge + * @return the registered gauge + */ + Gauge gauge(String name, Gauge gauge); + + // ----------------------------------------------------------------------------------------------------------------- + // Groups + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Creates a new {@link org.apache.flink.metrics.MetricGroup} and adds it to this groups sub-groups. + * + * @param name name of the group + * @return the created group + */ + MetricGroup addGroup(int name); + + /** + * Creates a new {@link org.apache.flink.metrics.MetricGroup} and adds it to this groups sub-groups. + * + * @param name name of the group + * @return the created group + */ + MetricGroup addGroup(String name); +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java new file mode 100644 index 0000000000000..0e8b0d5bfecd6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.AbstractMetricGroup; +import org.apache.flink.metrics.groups.Scope; +import org.apache.flink.metrics.reporter.JMXReporter; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.TimerTask; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; +import static org.apache.flink.metrics.groups.OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR; +import static org.apache.flink.metrics.groups.TaskManagerMetricGroup.DEFAULT_SCOPE_TM; +import static org.apache.flink.metrics.groups.TaskMetricGroup.DEFAULT_SCOPE_TASK; + +/** + * A MetricRegistry keeps track of all registered {@link org.apache.flink.metrics.Metric}s. It serves as the + * connection between {@link org.apache.flink.metrics.MetricGroup}s and {@link org.apache.flink.metrics.reporter.MetricReporter}s. + */ +@Internal +public class MetricRegistry { + private static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); + + private final MetricReporter reporter; + private java.util.Timer timer; + + public static final String KEY_METRICS_REPORTER_CLASS = "metrics.reporter.class"; + public static final String KEY_METRICS_REPORTER_ARGUMENTS = "metrics.reporter.arguments"; + public static final String KEY_METRICS_REPORTER_INTERVAL = "metrics.reporter.interval"; + + public static final String KEY_METRICS_SCOPE_NAMING_TM = "metrics.scope.tm"; + public static final String KEY_METRICS_SCOPE_NAMING_JOB = "metrics.scope.job"; + public static final String KEY_METRICS_SCOPE_NAMING_TASK = "metrics.scope.task"; + public static final String KEY_METRICS_SCOPE_NAMING_OPERATOR = "metrics.scope.operator"; + + private final Scope.ScopeFormat scopeConfig; + + /** + * Creates a new {@link MetricRegistry} and starts the configured reporter. + */ + public MetricRegistry(Configuration config) { + try { + String className = config.getString(KEY_METRICS_REPORTER_CLASS, null); + if (className == null) { + LOG.info("No reporter class name defined in flink-conf.yaml, defaulting to " + JMXReporter.class.getName() + "."); + className = JMXReporter.class.getName(); + } + + this.scopeConfig = createScopeConfig(config); + + Configuration reporterConfig = createReporterConfig(config); + Class reporterClass = Class.forName(className); + reporter = (MetricReporter) reporterClass.newInstance(); + reporter.open(reporterConfig); + + if (reporter instanceof Scheduled) { + String[] interval = config.getString(KEY_METRICS_REPORTER_INTERVAL, "10 SECONDS").split(" "); + long millis = TimeUnit.valueOf(interval[1]).toMillis(Long.parseLong(interval[0])); + timer = new java.util.Timer(true); + timer.schedule(new TimerTask() { + @Override + public void run() { + ((Scheduled) reporter).report(); + } + }, millis, millis); + } + } catch (InstantiationException | ClassNotFoundException e) { + throw new RuntimeException("Error while instantiating reporter.", e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Implementation error.", e); + } + } + + private static Configuration createReporterConfig(Configuration config) { + String[] interval = config.getString(KEY_METRICS_REPORTER_INTERVAL, "10 SECONDS").split(" "); + + String[] arguments = config.getString(KEY_METRICS_REPORTER_ARGUMENTS, "").split(" "); + + Configuration reporterConfig = new Configuration(); + reporterConfig.setString("period", interval[0]); + reporterConfig.setString("timeunit", interval[1]); + + if (arguments.length > 1) { + for (int x = 0; x < arguments.length; x += 2) { + reporterConfig.setString(arguments[x].replace("--", ""), arguments[x + 1]); + } + } + return reporterConfig; + } + + private static Scope.ScopeFormat createScopeConfig(Configuration config) { + String tmFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TM, DEFAULT_SCOPE_TM); + String jobFormat = config.getString(KEY_METRICS_SCOPE_NAMING_JOB, DEFAULT_SCOPE_JOB); + String taskFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TASK, DEFAULT_SCOPE_TASK); + String operatorFormat = config.getString(KEY_METRICS_SCOPE_NAMING_OPERATOR, DEFAULT_SCOPE_OPERATOR); + + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(tmFormat); + format.setJobFormat(jobFormat); + format.setTaskFormat(taskFormat); + format.setOperatorFormat(operatorFormat); + return format; + } + + public Scope.ScopeFormat getScopeConfig() { + return this.scopeConfig; + } + + /** + * Shuts down this registry and the associated {@link org.apache.flink.metrics.reporter.MetricReporter}. + */ + public void shutdown() { + if (timer != null) { + timer.cancel(); + } + if (reporter != null) { + reporter.close(); + } + } + + /** + * Registers a new {@link org.apache.flink.metrics.Metric} with this registry. + * + * @param metric metric to register + * @param name name of the metric + * @param parent group that contains the metric + */ + public void register(Metric metric, String name, AbstractMetricGroup parent) { + String metricName = reporter.generateName(name, parent.generateScope()); + + + this.reporter.notifyOfAddedMetric(metric, metricName); + } + + /** + * Un-registers the given {@link org.apache.flink.metrics.Metric} with this registry. + * + * @param metric metric to un-register + * @param name name of the metric + * @param parent group that contains the metric + */ + public void unregister(Metric metric, String name, AbstractMetricGroup parent) { + String metricName = reporter.generateName(name, parent.generateScope()); + + this.reporter.notifyOfRemovedMetric(metric, metricName); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java new file mode 100644 index 0000000000000..373ac0945cb8a --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Abstract {@link org.apache.flink.metrics.MetricGroup} that contains key functionality for adding metrics and groups. + */ +@Internal +public abstract class AbstractMetricGroup implements MetricGroup { + private static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class); + protected final MetricRegistry registry; + + // all metrics that are directly contained in this group + protected final Map metrics = new HashMap<>(); + // all generic groups that are directly contained in this group + protected final Map groups = new HashMap<>(); + + public AbstractMetricGroup(MetricRegistry registry) { + this.registry = registry; + } + + @Override + public void close() { + for (MetricGroup group : groups.values()) { + group.close(); + } + this.groups.clear(); + for (Map.Entry metric : metrics.entrySet()) { + registry.unregister(metric.getValue(), metric.getKey(), this); + } + this.metrics.clear(); + } + + // ----------------------------------------------------------------------------------------------------------------- + // Scope + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Generates the full scope based on the default/configured format that applies to all metrics within this group. + * + * @return generated scope + */ + public abstract List generateScope(); + + /** + * Generates the full scope based on the given format that applies to all metrics within this group. + * + * @param format format string + * @return generated scope + */ + public abstract List generateScope(Scope.ScopeFormat format); + + // ----------------------------------------------------------------------------------------------------------------- + // Metrics + // ----------------------------------------------------------------------------------------------------------------- + + @Override + public Counter counter(int name) { + return counter("" + name); + } + + @Override + public Counter counter(String name) { + Counter counter = new Counter(); + addMetric(name, counter); + return counter; + } + + @Override + public Gauge gauge(int name, Gauge gauge) { + return gauge("" + name, gauge); + } + + @Override + public Gauge gauge(String name, Gauge gauge) { + addMetric(name, gauge); + return gauge; + } + + protected MetricGroup addMetric(String name, Metric metric) { + if (!name.matches("[a-zA-Z0-9]*")) { + throw new IllegalArgumentException("Metric names may not contain special characters."); + } + if (metrics.containsKey(name)) { + LOG.warn("Detected metric name collision. This group already contains a group for the given group name. " + + this.generateScope().toString() + "." + name); + } + if (groups.containsKey(name)) { + LOG.warn("Detected metric name collision. This group already contains a group for the given metric name." + + this.generateScope().toString() + ")." + name); + } + metrics.put(name, metric); + registry.register(metric, name, this); + return this; + } + + // ----------------------------------------------------------------------------------------------------------------- + // Groups + // ----------------------------------------------------------------------------------------------------------------- + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + if (metrics.containsKey(name)) { + LOG.warn("Detected metric name collision. This group already contains a metric for the given group name." + + this.generateScope().toString() + "." + name); + } + if (!groups.containsKey(name)) { + groups.put(name, new GenericMetricGroup(registry, this, name)); + } + return groups.get(name); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java new file mode 100644 index 0000000000000..81851e2e767f4 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.metrics.groups.Scope.SCOPE_WILDCARD; + +/** + * Abstract {@link org.apache.flink.metrics.MetricGroup} that contains key functionality for modifying the scope. + */ +@Internal +public abstract class ComponentMetricGroup extends AbstractMetricGroup { + private final ComponentMetricGroup parent; + private final String format; + + + // Map: scope variable -> specific value + protected final Map formats; + + /** + * Creates a new ComponentMetricGroup. + * + * @param registry registry to register new metrics with + * @param parentGroup parent group, may be null + * @param scopeFormat default format string + */ + public ComponentMetricGroup(MetricRegistry registry, ComponentMetricGroup parentGroup, String scopeFormat) { + super(registry); + this.formats = new HashMap<>(); + this.parent = parentGroup; + this.format = scopeFormat; + } + + @Override + public List generateScope() { + return this.generateScope(this.format); + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + return generateScope(getScopeFormat(format)); + } + + protected abstract String getScopeFormat(Scope.ScopeFormat format); + + private List generateScope(String format) { + String[] components = Scope.split(format); + + List scope = new ArrayList<>(); + if (components[0].equals(SCOPE_WILDCARD)) { + if (this.parent != null) { + scope = this.parent.generateScope(); + } + this.replaceFormats(components); + addToList(scope, components, 1); + } else { + if (this.parent != null) { + this.parent.replaceFormats(components); + } + this.replaceFormats(components); + addToList(scope, components, 0); + } + return scope; + } + + private void replaceFormats(String[] components) { + if (this.parent != null) { + this.parent.replaceFormats(components); + } + for (int x = 0; x < components.length; x++) { + if (components[x].startsWith("<")) { + if (this.formats.containsKey(components[x])) { + components[x] = this.formats.get(components[x]); + } + } + } + } + + /** + * Adds all elements from the given array, starting from the given index, to the given list. + * + * @param list destination + * @param array source + * @param startIndex array index to start from + */ + private static void addToList(List list, String[] array, int startIndex) { + for (int x = startIndex; x < array.length; x++) { + list.add(array[x]); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java new file mode 100644 index 0000000000000..5886312aca647 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.List; + +/** + * A simple named {@link org.apache.flink.metrics.MetricGroup} with no special properties. + */ +@Internal +public class GenericMetricGroup extends AbstractMetricGroup { + private final AbstractMetricGroup parent; + + private final String name; + + protected GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, int name) { + this(registry, parent, "" + name); + } + + protected GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, String name) { + super(registry); + this.parent = parent; + this.name = name; + } + + @Override + public List generateScope() { + List scope = parent.generateScope(); + scope.add(name); + return scope; + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + List scope = parent.generateScope(format); + scope.add(name); + return scope; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java new file mode 100644 index 0000000000000..68d91c448712b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.List; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} that contains shareable pre-defined IO-related metrics. + */ +public class IOMetricGroup extends AbstractMetricGroup { + private final TaskMetricGroup parent; + + private transient final Counter numBytesIn; + private transient final Counter numBytesOut; + private transient final Counter numRecordsIn; + private transient final Counter numRecordsOut; + + public IOMetricGroup(MetricRegistry registry, TaskMetricGroup parent) { + super(registry); + this.parent = parent; + this.numBytesIn = parent.counter("numBytesIn"); + this.numBytesOut = parent.counter("numBytesOut"); + this.numRecordsIn = parent.counter("numRecordsIn"); + this.numRecordsOut = parent.counter("numRecordsOut"); + } + + @Override + public List generateScope() { + return parent.generateScope(); + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + return parent.generateScope(format); + } + + public Counter getBytesInCounter() { + return this.numBytesIn; + } + + public Counter getBytesOutCounter() { + return this.numBytesOut; + } + + public Counter getRecordsInCounter() { + return this.numRecordsIn; + } + + public Counter getRecordsOutCounter() { + return this.numRecordsOut; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java new file mode 100644 index 0000000000000..35a01f8d30e0c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.metrics.groups.TaskManagerMetricGroup.DEFAULT_SCOPE_TM; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing a Job. + * tasks = new HashMap<>(); + + public JobMetricGroup(MetricRegistry registry, TaskManagerMetricGroup taskManager, JobID id, String name) { + super(registry, taskManager, registry.getScopeConfig().getJobFormat()); + this.formats.put(SCOPE_JOB_ID, id.toString()); + this.formats.put(SCOPE_JOB_NAME, name); + } + + public TaskMetricGroup addTask(AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { + TaskMetricGroup task = new TaskMetricGroup(this.registry, this, id, attemptID, subtaskIndex, name); + tasks.put(id, task); + return task; + } + + @Override + public void close() { + super.close(); + for (MetricGroup group : tasks.values()) { + group.close(); + } + tasks.clear(); + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getJobFormat(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java new file mode 100644 index 0000000000000..6475eecc31a8d --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricRegistry; + +import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing an Operator. + */ +@Internal +public class OperatorMetricGroup extends ComponentMetricGroup { + public static final String SCOPE_OPERATOR_DESCRIPTOR = "operator"; + public static final String SCOPE_OPERATOR_NAME = Scope.format("operator_name"); + public static final String SCOPE_OPERATOR_SUBTASK_INDEX = Scope.format("subtask_index"); + public static final String DEFAULT_SCOPE_OPERATOR_COMPONENT = Scope.concat(SCOPE_OPERATOR_NAME, SCOPE_OPERATOR_SUBTASK_INDEX); + public static final String DEFAULT_SCOPE_OPERATOR = Scope.concat(DEFAULT_SCOPE_JOB, DEFAULT_SCOPE_OPERATOR_COMPONENT); + + protected OperatorMetricGroup(MetricRegistry registry, TaskMetricGroup task, String name, int subTaskIndex) { + super(registry, task, registry.getScopeConfig().getOperatorFormat()); + this.formats.put(SCOPE_OPERATOR_NAME, name); + this.formats.put(SCOPE_OPERATOR_SUBTASK_INDEX, "" + subTaskIndex); + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getOperatorFormat(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java new file mode 100644 index 0000000000000..47bae37ec4641 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; + +/** + * This class provides utility-functions for handling scopes. + */ +@Internal +public class Scope { + public static final String SCOPE_WILDCARD = "*"; + + private static final String SCOPE_PREFIX = "<"; + private static final String SCOPE_SUFFIX = ">"; + private static final String SCOPE_SPLIT = "."; + + private Scope() { + } + + /** + * Modifies the given string to resemble a scope variable. + * + * @param scope string to format + * @return formatted string + */ + public static String format(String scope) { + return SCOPE_PREFIX + scope + SCOPE_SUFFIX; + } + + /** + * Joins the given components into a single scope. + * + * @param components components to join + * @return joined scoped + */ + public static String concat(String... components) { + StringBuilder sb = new StringBuilder(); + sb.append(components[0]); + for (int x = 1; x < components.length; x++) { + sb.append(SCOPE_SPLIT); + sb.append(components[x]); + } + return sb.toString(); + } + + /** + * Splits the given scope into it's individual components. + * + * @param scope scope to split + * @return array of components + */ + public static String[] split(String scope) { + return scope.split("\\" + SCOPE_SPLIT); + } + + /** + * Simple container for component scope format strings. + */ + public static class ScopeFormat { + private String operatorFormat = OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR; + private String taskFormat = TaskMetricGroup.DEFAULT_SCOPE_TASK; + private String jobFormat = JobMetricGroup.DEFAULT_SCOPE_JOB; + private String taskManagerFormat = TaskManagerMetricGroup.DEFAULT_SCOPE_TM; + + public ScopeFormat() { + } + + public ScopeFormat setOperatorFormat(String format) { + this.operatorFormat = format; + return this; + } + + public ScopeFormat setTaskFormat(String format) { + this.taskFormat = format; + return this; + } + + public ScopeFormat setJobFormat(String format) { + this.jobFormat = format; + return this; + } + + public ScopeFormat setTaskManagerFormat(String format) { + this.taskManagerFormat = format; + return this; + } + + public String getOperatorFormat() { + return this.operatorFormat; + } + + public String getTaskFormat() { + return this.taskFormat; + } + + public String getJobFormat() { + return this.jobFormat; + } + + public String getTaskManagerFormat() { + return this.taskManagerFormat; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java new file mode 100644 index 0000000000000..e199ca7ea76c1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; + +import java.util.HashMap; +import java.util.Map; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing a TaskManager. + * jobs = new HashMap<>(); + + public TaskManagerMetricGroup(MetricRegistry registry, String host, String id) { + super(registry, null, registry.getScopeConfig().getTaskManagerFormat()); + this.formats.put(SCOPE_TM_HOST, host); + this.formats.put(SCOPE_TM_ID, id); + } + + public JobMetricGroup addJob(JobID id, String name) { + JobMetricGroup task = new JobMetricGroup(this.registry, this, id, name); + jobs.put(id, task); + return task; + } + + @Override + public void close() { + super.close(); + for (MetricGroup group : jobs.values()) { + group.close(); + } + jobs.clear(); + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getTaskManagerFormat(); + } +} + diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java new file mode 100644 index 0000000000000..4f8e010a528f7 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; + +/** + * Special {@link org.apache.flink.metrics.MetricGroup} representing a Task. + * operators = new HashMap<>(); + private IOMetricGroup ioMetrics; + + protected TaskMetricGroup(MetricRegistry registry, JobMetricGroup job, AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { + super(registry, job, registry.getScopeConfig().getTaskFormat()); + this.formats.put(SCOPE_TASK_ID, id.toString()); + this.formats.put(SCOPE_TASK_ATTEMPT, attemptID.toString()); + this.formats.put(SCOPE_TASK_NAME, name); + this.formats.put(SCOPE_TASK_SUBTASK_INDEX, "" + subtaskIndex); + this.subtaskIndex = subtaskIndex; + this.ioMetrics = new IOMetricGroup(registry, this); + } + + @Override + public void close() { + super.close(); + for (MetricGroup group : operators.values()) { + group.close(); + } + operators.clear(); + } + + public OperatorMetricGroup addOperator(String name) { + OperatorMetricGroup operator = new OperatorMetricGroup(this.registry, this, name, this.subtaskIndex); + operators.put(name, operator); + return operator; + } + + /** + * Returns the IOMetricGroup for this task. + * + * @return IOMetricGroup for this task. + */ + public IOMetricGroup getIOMetricGroup() { + return this.ioMetrics; + } + + @Override + protected String getScopeFormat(Scope.ScopeFormat format) { + return format.getTaskFormat(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java new file mode 100644 index 0000000000000..43f09b2293f16 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.reporter; + +import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; + +import java.util.Map; + +public abstract class AbstractReporter implements MetricReporter { + protected Map gauges = new ConcurrentHashMap<>(); + protected Map counters = new ConcurrentHashMap<>(); + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + if (metric instanceof Counter) { + counters.put(name, (Counter) metric); + } else if (metric instanceof Gauge) { + gauges.put(name, (Gauge) metric); + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + if (metric instanceof Counter) { + counters.remove(name); + } else if (metric instanceof Gauge) { + gauges.remove(name); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java new file mode 100644 index 0000000000000..0b2efe4b5de4e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.reporter; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.InstanceAlreadyExistsException; +import javax.management.InstanceNotFoundException; +import javax.management.MBeanRegistrationException; +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.NotCompliantMBeanException; +import javax.management.ObjectName; +import java.lang.management.ManagementFactory; +import java.util.List; + +/** + * {@link org.apache.flink.metrics.reporter.MetricReporter} that exports {@link org.apache.flink.metrics.Metric}s via JMX. + * + * Largely based on the JmxReporter class of the dropwizard metrics library + * https://github.com/dropwizard/metrics/blob/master/metrics-core/src/main/java/io/dropwizard/metrics/JmxReporter.java + */ +@Internal +public class JMXReporter implements MetricReporter { + private static final Logger LOG = LoggerFactory.getLogger(JMXReporter.class); + + private MBeanServer mBeanServer; + + private static final String PREFIX = "org.apache.flink.metrics:"; + private static final String KEY_PREFIX = "key"; + + public JMXReporter() { + this.mBeanServer = ManagementFactory.getPlatformMBeanServer(); + } + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + AbstractBean jmxMetric; + ObjectName jmxName; + try { + jmxName = new ObjectName(name); + } catch (MalformedObjectNameException e) { + throw new IllegalArgumentException("Metric name did not conform to JMX ObjectName rules: " + name, e); + } + + if (metric instanceof Gauge) { + jmxMetric = new JmxGauge((Gauge) metric); + } else if (metric instanceof Counter) { + jmxMetric = new JmxCounter((Counter) metric); + } else { + throw new IllegalArgumentException("Unknown metric type: " + metric.getClass()); + } + + try { + mBeanServer.registerMBean(jmxMetric, jmxName); + } catch (NotCompliantMBeanException e) { //implementation error on our side + LOG.error("Metric did not comply with JMX MBean naming rules.", e); + } catch (InstanceAlreadyExistsException e) { + LOG.error("A metric with the name " + jmxName + " was already registered.", e); + } catch (MBeanRegistrationException e) { + LOG.error("Failed to register metric.", e); + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + try { + mBeanServer.unregisterMBean(new ObjectName(name)); + } catch (MBeanRegistrationException e) { + LOG.error("Un-registering metric failed.", e); + } catch (MalformedObjectNameException e) { + LOG.error("Un-registering metric failed due to invalid name.", e); + } catch (InstanceNotFoundException e) { + //alright then + } + } + + @Override + public void open(Configuration config) { + } + + @Override + public void close() { + } + + @Override + public String generateName(String name, List origin) { + StringBuilder fullName = new StringBuilder(); + + fullName.append(PREFIX); + for (int x = 0; x < origin.size(); x++) { + fullName.append(KEY_PREFIX); + fullName.append(x); + fullName.append("="); + String value = origin.get(x); + value = value.replaceAll("\"", ""); + value = value.replaceAll(" ", "_"); + value = value.replaceAll("[,=;:?'*]", "-"); + fullName.append(value); + fullName.append(","); + } + fullName.append("name=" + name); + + return fullName.toString(); + } + + public interface MetricMBean { + } + + private abstract static class AbstractBean implements MetricMBean { + } + + public interface JmxCounterMBean extends MetricMBean { + long getCount(); + } + + private static class JmxCounter extends AbstractBean implements JmxCounterMBean { + private Counter counter; + + public JmxCounter(Counter counter) { + this.counter = counter; + } + + @Override + public long getCount() { + return counter.getCount(); + } + } + + public interface JmxGaugeMBean extends MetricMBean { + Object getValue(); + } + + private static class JmxGauge extends AbstractBean implements JmxGaugeMBean { + private final Gauge gauge; + + public JmxGauge(Gauge gauge) { + this.gauge = gauge; + } + + @Override + public Object getValue() { + return gauge.getValue(); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java new file mode 100644 index 0000000000000..2bca606e215f1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.reporter; + +import com.codahale.metrics.Reporter; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; + +import java.util.List; + +/** + * Reporters are used to export {@link org.apache.flink.metrics.Metric}s to an external backend. + *

+ * Reporters are instantiated generically and must have a no-argument constructor. + */ +@PublicEvolving +public interface MetricReporter extends Reporter { + /** + * Configures this reporter. Since reporters are instantiated generically and hence parameter-less, + * this method is the place where the reporters set their basic fields based on configuration values. + *

+ * This method is always called first on a newly instantiated reporter. + * + * @param config The configuration with all parameters. + */ + void open(Configuration config); + + /** + * Closes this reporter. Should be used to close channels, streams and release resources. + */ + void close(); + + /** + * Called when a new {@link org.apache.flink.metrics.Metric} was added. + * + * @param metric metric that was added + * @param name name of the metric + */ + void notifyOfAddedMetric(Metric metric, String name); + + /** + * Called when a {@link org.apache.flink.metrics.Metric} was removed. + * + * @param metric metric that was removed + * @param name name of the metric + */ + void notifyOfRemovedMetric(Metric metric, String name); + + /** + * Generates the reported name of a metric based on it's hierarchy/scope and associated name. + * + * @param name name of the metric + * @param scope hierarchy/scope of the metric + * @return reported name + */ + String generateName(String name, List scope); +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java new file mode 100644 index 0000000000000..3638f7a5308a7 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/Scheduled.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.reporter; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Marker interface for reporters that actively send out data periodically. + */ +@PublicEvolving +public interface Scheduled { + /** + * Report the current measurements. + * This method is called in regular intervals + */ + void report(); +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java index 858bc49403b15..83c88cc3608ae 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java @@ -31,6 +31,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; @@ -41,7 +42,7 @@ public class RuntimeUDFContextTest { @Test public void testBroadcastVariableNotFound() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(),new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(),new HashMap>(), new DummyMetricGroup()); try { ctx.getBroadcastVariable("some name"); @@ -71,7 +72,7 @@ public void testBroadcastVariableNotFound() { @Test public void testBroadcastVariableSimple() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name1", Arrays.asList(1, 2, 3, 4)); ctx.setBroadcastVariable("name2", Arrays.asList(1.0, 2.0, 3.0, 4.0)); @@ -105,7 +106,7 @@ public void testBroadcastVariableSimple() { @Test public void testBroadcastVariableWithInitializer() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); @@ -130,7 +131,7 @@ public void testBroadcastVariableWithInitializer() { @Test public void testResetBroadcastVariableWithInitializer() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); @@ -153,7 +154,7 @@ public void testResetBroadcastVariableWithInitializer() { @Test public void testBroadcastVariableWithInitializerAndMismatch() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>()); + RuntimeUDFContext ctx = new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java index 7ea007174c1c3..554820e981f4d 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/RichInputFormatTest.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.types.Value; import org.junit.Assert; import org.junit.Test; @@ -40,7 +41,7 @@ public class RichInputFormatTest { public void testCheckRuntimeContextAccess() { final SerializedInputFormat inputFormat = new SerializedInputFormat(); final TaskInfo taskInfo = new TaskInfo("test name", 1, 3, 0); - inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>())); + inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup())); Assert.assertEquals(inputFormat.getRuntimeContext().getIndexOfThisSubtask(), 1); Assert.assertEquals(inputFormat.getRuntimeContext().getNumberOfParallelSubtasks(),3); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java index 273f4f54e7389..09db3a99efff7 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/RichOutputFormatTest.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.types.Value; import org.junit.Assert; import org.junit.Test; @@ -41,7 +42,7 @@ public class RichOutputFormatTest { public void testCheckRuntimeContextAccess() { final SerializedOutputFormat inputFormat = new SerializedOutputFormat(); final TaskInfo taskInfo = new TaskInfo("test name", 1, 3, 0); - inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>())); + inputFormat.setRuntimeContext(new RuntimeUDFContext(taskInfo, getClass().getClassLoader(), new ExecutionConfig(), new HashMap>(), new HashMap>(), new DummyMetricGroup())); Assert.assertEquals(inputFormat.getRuntimeContext().getIndexOfThisSubtask(), 1); Assert.assertEquals(inputFormat.getRuntimeContext().getNumberOfParallelSubtasks(),3); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java index 5ca4c4cfe2d85..7c905c186d878 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSinkBaseTest.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.operators.util.TestRichOutputFormat; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.types.Nothing; import org.junit.Test; @@ -94,13 +95,13 @@ public void testDataSourceWithRuntimeContext() { final TaskInfo taskInfo = new TaskInfo("test_sink", 0, 1, 0); executionConfig.disableObjectReuse(); in.reset(); - sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(out.output, asList(TestIOData.RICH_NAMES)); executionConfig.enableObjectReuse(); out.clear(); in.reset(); - sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + sink.executeOnCollections(asList(TestIOData.NAMES), new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(out.output, asList(TestIOData.RICH_NAMES)); } catch(Exception e){ e.printStackTrace(); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java index 083039af10f71..c360c62fdbb5a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/GenericDataSourceBaseTest.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.operators.util.TestRichInputFormat; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; import java.util.HashMap; @@ -83,7 +84,7 @@ public void testDataSourceWithRuntimeContext() { executionConfig.disableObjectReuse(); assertEquals(false, in.hasBeenClosed()); assertEquals(false, in.hasBeenOpened()); - List resultMutableSafe = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultMutableSafe = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(true, in.hasBeenClosed()); assertEquals(true, in.hasBeenOpened()); @@ -91,7 +92,7 @@ public void testDataSourceWithRuntimeContext() { executionConfig.enableObjectReuse(); assertEquals(false, in.hasBeenClosed()); assertEquals(false, in.hasBeenOpened()); - List resultRegular = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultRegular = source.executeOnCollections(new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(true, in.hasBeenClosed()); assertEquals(true, in.hasBeenOpened()); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java index cda324526fe0c..9447efd865ec6 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; @@ -78,7 +79,7 @@ private void testExecuteOnCollection(FlatMapFunction udf, List result = getTestFlatMapOperator(udf) - .executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + .executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); Assert.assertEquals(input.size(), result.size()); Assert.assertEquals(input, result); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java index d119fe22e6a7b..a610a4dc5ffaf 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Test; @@ -125,9 +126,9 @@ public void join(String first, String second, Collector out) throws Exc ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(expected, resultSafe); assertEquals(expected, resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java index 6059ab1e438d5..7ecdefa83b1e0 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java @@ -39,6 +39,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; @SuppressWarnings("serial") @@ -112,9 +113,9 @@ public void close() throws Exception { final TaskInfo taskInfo = new TaskInfo(taskName, 0, 1, 0); ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap), executionConfig); + List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulatorMap, new DummyMetricGroup()), executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java index 71486a51bf1f6..50127185af76c 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java @@ -31,6 +31,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.api.common.functions.RichMapPartitionFunction; @@ -85,9 +86,9 @@ public void close() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java new file mode 100644 index 0000000000000..f7502e58dbb9b --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.metrics.util.TestReporter; +import org.junit.Assert; +import org.junit.Test; + +public class MetricGroupTest { + /** + * Verifies that group methods instantiate the correct metric with the given name. + */ + @Test + public void testMetricInstantiation() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter1.class.getName()); + + MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); + + root.counter("counter"); + Assert.assertTrue(TestReporter1.lastPassedMetric instanceof Counter); + Assert.assertEquals("counter", TestReporter1.lastPassedName); + + root.gauge("gauge", new Gauge() { + @Override + public Object getValue() { + return null; + } + }); + Assert.assertTrue(TestReporter1.lastPassedMetric instanceof Gauge); + Assert.assertEquals("gauge", TestReporter1.lastPassedName); + } + + protected static class TestReporter1 extends TestReporter { + public static Metric lastPassedMetric; + public static String lastPassedName; + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + lastPassedMetric = metric; + lastPassedName = name; + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + } + } + + /** + * Verifies that metric names containing special characters are rejected. + */ + @Test(expected = IllegalArgumentException.class) + public void testInvalidMetricName() { + Configuration config = new Configuration(); + + MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); + root.counter("=)(/!"); + } + + /** + * Verifies that when attempting to create a group with the name of an existing one the existing one will be returned instead. + */ + @Test + public void testDuplicateGroupName() { + Configuration config = new Configuration(); + + MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); + + MetricGroup group1 = root.addGroup("group"); + MetricGroup group2 = root.addGroup("group"); + MetricGroup group3 = root.addGroup("group"); + Assert.assertTrue(group1 == group2 && group2 == group3); + + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java b/flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java new file mode 100644 index 0000000000000..32cc11ce940a8 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/MetricRegistryTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.JobMetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.Scope; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.reporter.Scheduled; +import org.apache.flink.metrics.util.TestReporter; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class MetricRegistryTest { + /** + * Verifies that the reporter class argument is correctly used to instantiate and open the reporter. + */ + @Test + public void testReporterInstantiation() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter1.class.getName()); + + new MetricRegistry(config); + + Assert.assertTrue(TestReporter1.wasOpened); + } + + protected static class TestReporter1 extends TestReporter { + public static boolean wasOpened = false; + + @Override + public void open(Configuration config) { + wasOpened = true; + } + } + + /** + * Verifies that configured arguments are properly forwarded to the reporter. + */ + @Test + public void testReporterArgumentForwarding() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter2.class.getName()); + config.setString(MetricRegistry.KEY_METRICS_REPORTER_ARGUMENTS, "--arg1 hello --arg2 world"); + + new MetricRegistry(config); + } + + protected static class TestReporter2 extends TestReporter { + @Override + public void open(Configuration config) { + Assert.assertEquals("hello", config.getString("arg1", null)); + Assert.assertEquals("world", config.getString("arg2", null)); + } + } + + /** + * Verifies that reporters implementing the Scheduled interface are regularly called to report the metrics. + * + * @throws InterruptedException + */ + @Test + public void testReporterScheduling() throws InterruptedException { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter3.class.getName()); + config.setString(MetricRegistry.KEY_METRICS_REPORTER_INTERVAL, "50 MILLISECONDS"); + + new MetricRegistry(config); + + long start = System.currentTimeMillis(); + for (int x = 0; x < 10; x++) { + Thread.sleep(100); + int reportCount = TestReporter3.reportCount; + long curT = System.currentTimeMillis(); + /** + * Within a given time-frame T only T/500 reports may be triggered due to the interval between reports. + * This value however does not not take the first triggered report into account (=> +1). + * Furthermore we have to account for the mis-alignment between reports being triggered and our time + * measurement (=> +1); for T=200 a total of 4-6 reports may have been + * triggered depending on whether the end of the interval for the first reports ends before + * or after T=50. + */ + long maxAllowedReports = (curT - start) / 50 + 2; + Assert.assertTrue("Too many report were triggered.", maxAllowedReports >= reportCount); + } + Assert.assertTrue("No report was triggered.", TestReporter3.reportCount > 0); + } + + protected static class TestReporter3 extends TestReporter implements Scheduled { + public static int reportCount = 0; + + @Override + public void report() { + reportCount++; + } + } + + /** + * Verifies that groups are correctly created, nesting works, and names are properly forwarded to generate names. + */ + @Test + public void testMetricGroupGeneration() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter4.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + MetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); + root.counter("rootCounter"); + root.addGroup("top").counter("topCounter"); + } + + protected static class TestReporter4 extends TestReporter { + @Override + public String generateName(String name, List scope) { + if (name.compareTo("rootCounter") == 0) { + Assert.assertEquals("host", scope.get(0)); + return "success"; + } else if (name.compareTo("topCounter") == 0) { + Assert.assertEquals("host", scope.get(0)); + Assert.assertEquals("taskmanager", scope.get(1)); + return "success"; + } else { + Assert.fail(); + return null; + } + } + } + + /** + * Verifies that reporters implementing the Listener interface are notified when Metrics are added or removed. + */ + @Test + public void testListener() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter6.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + MetricGroup root = new TaskManagerMetricGroup(registry, "host", "id"); + root.counter("rootCounter"); + root.close(); + + Assert.assertTrue(TestReporter6.addCalled); + Assert.assertTrue(TestReporter6.removeCalled); + } + + protected static class TestReporter6 extends TestReporter { + public static boolean addCalled = false; + public static boolean removeCalled = false; + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + addCalled = true; + Assert.assertTrue(metric instanceof Counter); + Assert.assertEquals("rootCounter", name); + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + removeCalled = true; + Assert.assertTrue(metric instanceof Counter); + Assert.assertEquals("rootCounter", name); + } + } + + /** + * Verifies that the scope configuration is properly extracted. + */ + @Test + public void testScopeConfig() { + Configuration config = new Configuration(); + + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_TM, "A"); + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_JOB, "B"); + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_TASK, "C"); + config.setString(MetricRegistry.KEY_METRICS_SCOPE_NAMING_OPERATOR, "D"); + + Scope.ScopeFormat scopeConfig = new MetricRegistry(config).getScopeConfig(); + + Assert.assertEquals("A", scopeConfig.getTaskManagerFormat()); + Assert.assertEquals("B", scopeConfig.getJobFormat()); + Assert.assertEquals("C", scopeConfig.getTaskFormat()); + Assert.assertEquals("D", scopeConfig.getOperatorFormat()); + + Scope.ScopeFormat emptyScopeConfig = new MetricRegistry(new Configuration()).getScopeConfig(); + + Assert.assertEquals(TaskManagerMetricGroup.DEFAULT_SCOPE_TM, emptyScopeConfig.getTaskManagerFormat()); + Assert.assertEquals(JobMetricGroup.DEFAULT_SCOPE_JOB, emptyScopeConfig.getJobFormat()); + Assert.assertEquals(TaskMetricGroup.DEFAULT_SCOPE_TASK, emptyScopeConfig.getTaskFormat()); + Assert.assertEquals(OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR, emptyScopeConfig.getOperatorFormat()); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java new file mode 100644 index 0000000000000..89483b3dbcccf --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class JobGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job"); + + List scope = operator.generateScope(); + assertEquals(4, scope.size()); + assertEquals("job", scope.get(3)); + } + + @Test + public void testGenerateScopeWildcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setJobFormat(Scope.concat(Scope.SCOPE_WILDCARD, "superjob", JobMetricGroup.SCOPE_JOB_NAME)); + + List scope = operator.generateScope(format); + assertEquals(5, scope.size()); + assertEquals("superjob", scope.get(3)); + assertEquals("job", scope.get(4)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setJobFormat(Scope.concat(TaskManagerMetricGroup.SCOPE_TM_HOST, "superjob", JobMetricGroup.SCOPE_JOB_NAME)); + + List scope = operator.generateScope(format); + assertEquals(3, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("superjob", scope.get(1)); + assertEquals("job", scope.get(2)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java new file mode 100644 index 0000000000000..4f33d2a54edf0 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class OperatorGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + + List scope = operator.generateScope(); + assertEquals(6, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + assertEquals("job", scope.get(3)); + assertEquals("operator", scope.get(4)); + assertEquals("0", scope.get(5)); + } + + @Test + public void testGenerateScopeWildcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setOperatorFormat(Scope.concat(Scope.SCOPE_WILDCARD, "op", OperatorMetricGroup.SCOPE_OPERATOR_NAME)); + + List scope = operator.generateScope(format); + assertEquals(7, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + assertEquals("job", scope.get(3)); + assertEquals("task", scope.get(4)); + assertEquals("op", scope.get(5)); + assertEquals("operator", scope.get(6)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setOperatorFormat(Scope.concat("jobs", JobMetricGroup.SCOPE_JOB_NAME, "op", OperatorMetricGroup.SCOPE_OPERATOR_NAME)); + + List scope = operator.generateScope(format); + assertEquals(4, scope.size()); + assertEquals("jobs", scope.get(0)); + assertEquals("job", scope.get(1)); + assertEquals("op", scope.get(2)); + assertEquals("operator", scope.get(3)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java new file mode 100644 index 0000000000000..c49fdcdc95f1f --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TaskGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task"); + + List scope = operator.generateScope(); + assertEquals(5, scope.size()); + assertEquals("task", scope.get(4)); + } + + @Test + public void testGenerateScopeWilcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskFormat(Scope.concat(Scope.SCOPE_WILDCARD, "supertask", TaskMetricGroup.SCOPE_TASK_NAME)); + + List scope = operator.generateScope(format); + assertEquals(6, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + assertEquals("job", scope.get(3)); + assertEquals("supertask", scope.get(4)); + assertEquals("task", scope.get(5)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") + .addJob(new JobID(), "job") + .addTask(new AbstractID(), new AbstractID(), 0, "task"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskFormat(Scope.concat(TaskManagerMetricGroup.SCOPE_TM_HOST, JobMetricGroup.SCOPE_JOB_NAME, "supertask", TaskMetricGroup.SCOPE_TASK_NAME)); + + List scope = operator.generateScope(format); + assertEquals(4, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("job", scope.get(1)); + assertEquals("supertask", scope.get(2)); + assertEquals("task", scope.get(3)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java new file mode 100644 index 0000000000000..7b3286d05aacb --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.groups; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class TaskManagerGroupTest { + @Test + public void testGenerateScopeDefault() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskManagerMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id"); + + List scope = operator.generateScope(); + assertEquals(3, scope.size()); + assertEquals("host", scope.get(0)); + assertEquals("taskmanager", scope.get(1)); + assertEquals("id", scope.get(2)); + } + + @Test + public void testGenerateScopeWildcard() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskManagerMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(Scope.concat(Scope.SCOPE_WILDCARD, "superhost", TaskManagerMetricGroup.SCOPE_TM_HOST)); + + List scope = operator.generateScope(format); + assertEquals(2, scope.size()); + assertEquals("superhost", scope.get(0)); + assertEquals("host", scope.get(1)); + } + + @Test + public void testGenerateScopeCustom() { + MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskManagerMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id"); + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(Scope.concat("h", TaskManagerMetricGroup.SCOPE_TM_HOST, "t", TaskManagerMetricGroup.SCOPE_TM_ID)); + + List scope = operator.generateScope(format); + assertEquals(4, scope.size()); + assertEquals("h", scope.get(0)); + assertEquals("host", scope.get(1)); + assertEquals("t", scope.get(2)); + assertEquals("id", scope.get(3)); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java b/flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java new file mode 100644 index 0000000000000..0d683c25c9c99 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/reporter/JMXReporterTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.reporter; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class JMXReporterTest { + /** + * Verifies that the JMXReporter properly generates the JMX name. + */ + @Test + public void testGenerateName() { + String name = "metric"; + + List scope = new ArrayList<>(); + scope.add("value0"); + scope.add("value1"); + scope.add("\"value2 (test),=;:?'"); + + String jmxName = new JMXReporter().generateName(name, scope); + + Assert.assertEquals("org.apache.flink.metrics:key0=value0,key1=value1,key2=value2_(test)------,name=metric", jmxName); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java new file mode 100644 index 0000000000000..d607072e8fea2 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.JobMetricGroup; +import org.apache.flink.util.AbstractID; + +public class DummyJobMetricGroup extends JobMetricGroup { + public DummyJobMetricGroup() { + super(new DummyMetricRegistry(), new DummyTaskManagerMetricGroup(), new JobID(), "job"); + } + + @Override + public DummyTaskMetricGroup addTask(AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { + return new DummyTaskMetricGroup(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java new file mode 100644 index 0000000000000..26df8749348f9 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.AbstractMetricGroup; +import org.apache.flink.metrics.groups.Scope; + +import java.util.ArrayList; +import java.util.List; + +public class DummyMetricGroup extends AbstractMetricGroup { + public DummyMetricGroup() { + super(new DummyMetricRegistry()); + } + + @Override + public List generateScope() { + return new ArrayList<>(); + } + + @Override + public List generateScope(Scope.ScopeFormat format) { + return new ArrayList<>(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java new file mode 100644 index 0000000000000..f0d6d3f4cd780 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricRegistry; + +public class DummyMetricRegistry extends MetricRegistry { + private static final Configuration config; + + static { + config = new Configuration(); + config.setString(KEY_METRICS_REPORTER_CLASS, DummyReporter.class.getCanonicalName()); + } + + public DummyMetricRegistry() { + super(new Configuration()); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java new file mode 100644 index 0000000000000..eb45f6ae31cd4 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; + +public class DummyOperatorMetricGroup extends OperatorMetricGroup { + public DummyOperatorMetricGroup() { + super(new DummyMetricRegistry(), new DummyTaskMetricGroup(), "operator", 0); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java new file mode 100644 index 0000000000000..23a77683d993e --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyReporter.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.reporter.MetricReporter; + +import java.util.List; + +public class DummyReporter implements MetricReporter { + @Override + public void open(Configuration config) { + } + + @Override + public void close() { + } + + @Override + public void notifyOfAddedMetric(Metric metric, String name) { + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String name) { + } + + @Override + public String generateName(String name, List scope) { + return ""; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java new file mode 100644 index 0000000000000..1c7d33bc4db64 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.TaskManagerMetricGroup; + +public class DummyTaskManagerMetricGroup extends TaskManagerMetricGroup { + public DummyTaskManagerMetricGroup() { + super(new DummyMetricRegistry(), "host", "id"); + } + + public DummyJobMetricGroup addJob(JobID id, String name) { + return new DummyJobMetricGroup(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java new file mode 100644 index 0000000000000..53683f4f8430b --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.util.AbstractID; + +public class DummyTaskMetricGroup extends TaskMetricGroup { + public DummyTaskMetricGroup() { + super(new DummyMetricRegistry(), new DummyJobMetricGroup(), new AbstractID(), new AbstractID(), 0, "task"); + } + + public DummyOperatorMetricGroup addOperator(String name) { + return new DummyOperatorMetricGroup(); + } + + @Override + protected MetricGroup addMetric(String name, Metric metric) { + return this; + } + + @Override + public MetricGroup addGroup(int name) { + return addGroup("" + name); + } + + @Override + public MetricGroup addGroup(String name) { + return new DummyMetricGroup(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java b/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java new file mode 100644 index 0000000000000..482d1e8aa22c7 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.reporter.MetricReporter; + +import java.util.List; + +public class TestReporter extends AbstractReporter { + @Override + public void open(Configuration config) { + } + + @Override + public void close() { + } + + @Override + public String generateName(String name, List scope) { + return name; + } +} diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index ffbec07b71bab..b6bdbed805de7 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -103,6 +103,8 @@ KEY_ENV_SSH_OPTS="env.ssh.opts" KEY_RECOVERY_MODE="recovery.mode" KEY_ZK_HEAP_MB="zookeeper.heap.mb" +KEY_METRICS_JMX_PORT="metrics.jmx.port" + ######################################################################################################################## # PATHS AND CONFIG ######################################################################################################################## @@ -240,6 +242,10 @@ if [ -z "${RECOVERY_MODE}" ]; then RECOVERY_MODE=$(readFromConfig ${KEY_RECOVERY_MODE} "standalone" "${YAML_CONF}") fi +if [ -z "${JMX_PORT}" ]; then + JMX_PORT=$(readFromConfig ${KEY_METRICS_JMX_PORT} 9010 "${YAML_CONF}") +fi + # Arguments for the JVM. Used for job and task manager JVMs. # DO NOT USE FOR MEMORY SETTINGS! Use conf/flink-conf.yaml with keys # KEY_JOBM_MEM_SIZE and KEY_TASKM_MEM_SIZE for that! diff --git a/flink-dist/src/main/flink-bin/bin/flink-daemon.sh b/flink-dist/src/main/flink-bin/bin/flink-daemon.sh index 1ef743945443a..cc7163f254361 100644 --- a/flink-dist/src/main/flink-bin/bin/flink-daemon.sh +++ b/flink-dist/src/main/flink-bin/bin/flink-daemon.sh @@ -23,14 +23,24 @@ USAGE="Usage: flink-daemon.sh (start|stop|stop-all) (jobmanager|taskmanager|zook STARTSTOP=$1 DAEMON=$2 ARGS=("${@:3}") # get remaining arguments as array +JMX_ARGS="" + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +. "$bin"/config.sh case $DAEMON in (jobmanager) CLASS_TO_RUN=org.apache.flink.runtime.jobmanager.JobManager + if [ "${ARGS[3]}" == "local" ]; then + JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=${JMX_PORT} -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" + fi ;; (taskmanager) CLASS_TO_RUN=org.apache.flink.runtime.taskmanager.TaskManager + JMX_ARGS="-Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=${JMX_PORT} -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" ;; (zookeeper) @@ -43,11 +53,6 @@ case $DAEMON in ;; esac -bin=`dirname "$0"` -bin=`cd "$bin"; pwd` - -. "$bin"/config.sh - if [ "$FLINK_IDENT_STRING" = "" ]; then FLINK_IDENT_STRING="$USER" fi @@ -96,12 +101,13 @@ case $STARTSTOP in count="${#active[@]}" if [ ${count} -gt 0 ]; then + JMX_ARGS="" echo "[INFO] $count instance(s) of $DAEMON are already running on $HOSTNAME." fi fi echo "Starting $DAEMON daemon on host $HOSTNAME." - $JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} "${log_setting[@]}" -classpath "`manglePathList "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN} "${ARGS[@]}" > "$out" 2>&1 < /dev/null & + $JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} ${JMX_ARGS} "${log_setting[@]}" -classpath "`manglePathList "$FLINK_TM_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`" ${CLASS_TO_RUN} "${ARGS[@]}" > "$out" 2>&1 < /dev/null & mypid=$! diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java index 8ce83f3415665..31b3ba2af8cd2 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java @@ -31,6 +31,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Assert; import org.junit.Test; @@ -76,7 +77,7 @@ public void testExecuteOnCollection() { final HashMap> accumulators = new HashMap>(); final HashMap> cpTasks = new HashMap<>(); final TaskInfo taskInfo = new TaskInfo("Test UDF", 0, 4, 0); - final RuntimeContext ctx = new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulators); + final RuntimeContext ctx = new RuntimeUDFContext(taskInfo, null, executionConfig, cpTasks, accumulators, new DummyMetricGroup()); { SumCoGroup udf1 = new SumCoGroup(); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java index a5632812a8b7f..1b627c403ce15 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Test; @@ -168,9 +169,9 @@ public void close() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java index a6b3debf83be4..1d5668bd9cb8c 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.util.Collector; import org.junit.Test; @@ -107,9 +108,9 @@ public void join(Tuple3 first, Tuple2 final TaskInfo taskInfo = new TaskInfo("op", 0, 1, 0); ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); assertEquals(expected, new HashSet<>(resultSafe)); assertEquals(expected, new HashSet<>(resultRegular)); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java index c04916d0477ab..4317c03ffaf8f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.junit.Test; import java.util.ArrayList; @@ -145,9 +146,9 @@ public void close() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.disableObjectReuse(); - List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); executionConfig.enableObjectReuse(); - List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>()), executionConfig); + List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskInfo, null, executionConfig, new HashMap>(), new HashMap>(), new DummyMetricGroup()), executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); Set> resultSetRegular = new HashSet>(resultRegular); diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml new file mode 100644 index 0000000000000..84d97228c08f6 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/pom.xml @@ -0,0 +1,72 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-dropwizard + flink-metrics-dropwizard + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + + + + maven-assembly-plugin + 2.4 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java new file mode 100644 index 0000000000000..a7309be39aa9a --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.dropwizard; + +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.ScheduledReporter; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.dropwizard.metrics.CounterWrapper; +import org.apache.flink.dropwizard.metrics.GaugeWrapper; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.Scheduled; + +import java.util.List; + +/** + * Base class for {@link org.apache.flink.metrics.reporter.MetricReporter} that wraps a + * Dropwizard {@link com.codahale.metrics.Reporter}. + */ +@PublicEvolving +public abstract class ScheduledDropwizardReporter implements MetricReporter, Scheduled { + protected MetricRegistry registry; + protected ScheduledReporter reporter; + + public static final String ARG_HOST = "host"; + public static final String ARG_PORT = "port"; + public static final String ARG_PREFIX = "prefix"; + public static final String ARG_CONVERSION_RATE = "rateConversion"; + public static final String ARG_CONVERSION_DURATION = "durationConversion"; + + protected ScheduledDropwizardReporter() { + this.registry = new MetricRegistry(); + } + + @Override + public synchronized void notifyOfAddedMetric(Metric metric, String name) { + if (metric instanceof Counter) { + registry.register(name, new CounterWrapper((Counter) metric)); + } else if (metric instanceof Gauge) { + registry.register(name, new GaugeWrapper((Gauge) metric)); + } + } + + @Override + public synchronized void notifyOfRemovedMetric(Metric metric, String name) { + registry.remove(name); + } + + public abstract ScheduledReporter getReporter(Configuration config); + + @Override + public void open(Configuration config) { + this.reporter = getReporter(config); + } + + @Override + public void close() { + this.reporter.stop(); + } + + @Override + public String generateName(String name, List scope) { + StringBuilder sb = new StringBuilder(); + for (String s : scope) { + sb.append(s); + sb.append('.'); + } + sb.append(name); + return sb.toString(); + } + + @Override + public synchronized void report() { + this.reporter.report( + this.registry.getGauges(), + this.registry.getCounters(), + this.registry.getHistograms(), + this.registry.getMeters(), + this.registry.getTimers()); + } +} diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java new file mode 100644 index 0000000000000..f6630b9854125 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.dropwizard.metrics; + +import org.apache.flink.metrics.Counter; + +public class CounterWrapper extends com.codahale.metrics.Counter { + private final Counter counter; + + public CounterWrapper(Counter counter) { + this.counter = counter; + } + + @Override + public long getCount() { + return this.counter.getCount(); + } +} diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java new file mode 100644 index 0000000000000..d47090d8b36d5 --- /dev/null +++ b/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.dropwizard.metrics; + +import org.apache.flink.metrics.Gauge; + +public class GaugeWrapper implements com.codahale.metrics.Gauge { + private final Gauge gauge; + + public GaugeWrapper(Gauge gauge) { + this.gauge = gauge; + } + + @Override + public Object getValue() { + return this.gauge.getValue(); + } +} diff --git a/flink-metrics/flink-metrics-ganglia/pom.xml b/flink-metrics/flink-metrics-ganglia/pom.xml new file mode 100644 index 0000000000000..c4f51da9a2cec --- /dev/null +++ b/flink-metrics/flink-metrics-ganglia/pom.xml @@ -0,0 +1,90 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-ganglia + flink-metrics-ganglia + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + org.apache.flink + flink-metrics-dropwizard + ${project.version} + + + + info.ganglia.gmetric4j + gmetric4j + 1.0.7 + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + io.dropwizard.metrics + metrics-ganglia + ${metrics.version} + + + + + + + maven-assembly-plugin + 2.4 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java b/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java new file mode 100644 index 0000000000000..a1dafc9c71a7c --- /dev/null +++ b/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.graphite; + +import com.codahale.metrics.ScheduledReporter; +import info.ganglia.gmetric4j.gmetric.GMetric; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.dropwizard.ScheduledDropwizardReporter; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class GangliaReporter extends ScheduledDropwizardReporter { + public static final String ARG_DMAX = "dmax"; + public static final String ARG_TMAX = "tmax"; + public static final String ARG_TTL = "ttl"; + public static final String ARG_MODE_ADDRESSING = "addressingMode"; + + @Override + public ScheduledReporter getReporter(Configuration config) { + + try { + String host = config.getString(ARG_HOST, null); + int port = config.getInteger(ARG_PORT, -1); + if (host == null || host.length() == 0 || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + String addressingMode = config.getString(ARG_MODE_ADDRESSING, "MULTICAST"); + int ttl = config.getInteger(ARG_TTL, -1); + GMetric gMetric = new GMetric(host, port, GMetric.UDPAddressingMode.valueOf(addressingMode), ttl); + + String prefix = config.getString(ARG_PREFIX, null); + String conversionRate = config.getString(ARG_CONVERSION_RATE, null); + String conversionDuration = config.getString(ARG_CONVERSION_DURATION, null); + int dMax = config.getInteger(ARG_DMAX, 0); + int tMax = config.getInteger(ARG_TMAX, 60); + + com.codahale.metrics.ganglia.GangliaReporter.Builder builder = + com.codahale.metrics.ganglia.GangliaReporter.forRegistry(registry); + + if (prefix != null) { + builder.prefixedWith(prefix); + } + if (conversionRate != null) { + builder.convertRatesTo(TimeUnit.valueOf(conversionRate)); + } + if (conversionDuration != null) { + builder.convertDurationsTo(TimeUnit.valueOf(conversionDuration)); + } + builder.withDMax(dMax); + builder.withTMax(tMax); + + return builder.build(gMetric); + } catch (IOException e) { + throw new RuntimeException("Error while instantiating GangliaReporter.", e); + } + } +} diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml new file mode 100644 index 0000000000000..45fb01857df4c --- /dev/null +++ b/flink-metrics/flink-metrics-graphite/pom.xml @@ -0,0 +1,84 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-graphite + flink-metrics-graphite + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + org.apache.flink + flink-metrics-dropwizard + ${project.version} + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + + io.dropwizard.metrics + metrics-graphite + ${metrics.version} + + + + + + + maven-assembly-plugin + 2.4 + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + + diff --git a/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java b/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java new file mode 100644 index 0000000000000..b28d7a4904643 --- /dev/null +++ b/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.graphite; + +import com.codahale.metrics.ScheduledReporter; +import com.codahale.metrics.graphite.Graphite; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.dropwizard.ScheduledDropwizardReporter; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class GraphiteReporter extends ScheduledDropwizardReporter { + @Override + public ScheduledReporter getReporter(Configuration config) { + String host = config.getString(ARG_HOST, null); + int port = config.getInteger(ARG_PORT, -1); + + if (host == null || host.length() == 0 || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + + String prefix = config.getString(ARG_PREFIX, null); + String conversionRate = config.getString(ARG_CONVERSION_RATE, null); + String conversionDuration = config.getString(ARG_CONVERSION_DURATION, null); + + com.codahale.metrics.graphite.GraphiteReporter.Builder builder = + com.codahale.metrics.graphite.GraphiteReporter.forRegistry(registry); + + if (prefix != null) { + builder.prefixedWith(prefix); + } + + if (conversionRate != null) { + builder.convertRatesTo(TimeUnit.valueOf(conversionRate)); + } + + if (conversionDuration != null) { + builder.convertDurationsTo(TimeUnit.valueOf(conversionDuration)); + } + + return builder.build(new Graphite(host, port)); + } + + @Override + public String generateName(String name, List scope) { + StringBuilder sb = new StringBuilder(); + for (String s : scope) { + sb.append(s.replace(".", "_").replace("\"", "")); + sb.append("."); + } + sb.append(name); + return sb.toString(); + } +} diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml new file mode 100644 index 0000000000000..5d639087067d8 --- /dev/null +++ b/flink-metrics/flink-metrics-statsd/pom.xml @@ -0,0 +1,43 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-metrics + 1.1-SNAPSHOT + .. + + + flink-metrics-statsd + flink-metrics-statsd + + + + org.apache.flink + flink-core + ${project.version} + provided + + + diff --git a/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java b/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java new file mode 100644 index 0000000000000..288b8b86aa99b --- /dev/null +++ b/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.metrics.statsd; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.reporter.AbstractReporter; +import org.apache.flink.metrics.reporter.Scheduled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetSocketAddress; +import java.net.SocketException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * Largely based on the StatsDReporter class by ReadyTalk + * https://github.com/ReadyTalk/metrics-statsd/blob/master/metrics3-statsd/src/main/java/com/readytalk/metrics/StatsDReporter.java + * + * Ported since it was not present in maven central. + */ +public class StatsDReporter extends AbstractReporter implements Scheduled { + private static final Logger LOG = LoggerFactory.getLogger(StatsDReporter.class); + + public static final String ARG_HOST = "host"; + public static final String ARG_PORT = "port"; + public static final String ARG_CONVERSION_RATE = "rateConversion"; + public static final String ARG_CONVERSION_DURATION = "durationConversion"; + + private DatagramSocket socket; + private InetSocketAddress address; + + private double durationFactor; + private double rateFactor; + + @Override + public void open(Configuration config) { + String host = config.getString(ARG_HOST, null); + int port = config.getInteger(ARG_PORT, -1); + + if (host == null || host.length() == 0 || port < 1) { + throw new IllegalArgumentException("Invalid host/port configuration. Host: " + host + " Port: " + port); + } + + String conversionRate = config.getString(ARG_CONVERSION_RATE, "SECONDS"); + String conversionDuration = config.getString(ARG_CONVERSION_DURATION, "MILLISECONDS"); + + this.address = new InetSocketAddress(host, port); + this.rateFactor = TimeUnit.valueOf(conversionRate).toSeconds(1); + this.durationFactor = 1.0 / TimeUnit.valueOf(conversionDuration).toNanos(1); + try { + this.socket = new DatagramSocket(0); + } catch (SocketException e) { + throw new RuntimeException("Failure while creating socket. ", e); + } + } + + @Override + public void close() { + if (socket != null && !socket.isClosed()) { + socket.close(); + } + } + + @Override + public String generateName(String name, List scope) { + StringBuilder sb = new StringBuilder(); + for (String s : scope) { + sb.append(s); + sb.append('.'); + } + sb.append(name); + return sb.toString(); + } + + public void send(final String name, final double value) { + send(name, "" + value); + } + + public void send(final String name, final String value) { + try { + String formatted = String.format("%s:%s|g", name, value); + byte[] data = formatted.getBytes(); + socket.send(new DatagramPacket(data, data.length, this.address)); + } catch (IOException e) { + LOG.error("unable to send packet to statsd at '{}:{}'", address.getHostName(), address.getPort()); + } + } + + @Override + public void report() { + for (Map.Entry entry : gauges.entrySet()) { + reportGauge(entry.getKey(), entry.getValue()); + } + + for (Map.Entry entry : counters.entrySet()) { + reportCounter(entry.getKey(), entry.getValue()); + } + } + + private void reportCounter(final String name, final Counter counter) { + send(name, counter.getCount()); + } + + private void reportGauge(final String name, final Gauge gauge) { + final String value = gauge.getValue().toString(); + if (value != null) { + send((name), value); + } + } +} diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml new file mode 100644 index 0000000000000..542f49c26bc32 --- /dev/null +++ b/flink-metrics/pom.xml @@ -0,0 +1,42 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-parent + 1.1-SNAPSHOT + .. + + + flink-metrics + flink-metrics + pom + + + flink-metrics-dropwizard + flink-metrics-ganglia + flink-metrics-graphite + flink-metrics-statsd + + diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 13c4fa51d0c86..8007428c0073b 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -34,10 +34,6 @@ under the License. jar - - 3.1.0 - - org.apache.flink diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 2b1c224812877..d3ebc952b98a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -45,6 +45,7 @@ public final class TaskDeploymentDescriptor implements Serializable { /** The ID of the job the tasks belongs to. */ private final JobID jobID; + private final String jobName; /** The task's job vertex ID. */ private final JobVertexID vertexID; @@ -99,6 +100,7 @@ public final class TaskDeploymentDescriptor implements Serializable { */ public TaskDeploymentDescriptor( JobID jobID, + String jobName, JobVertexID vertexID, ExecutionAttemptID executionId, SerializedValue serializedExecutionConfig, @@ -123,6 +125,7 @@ public TaskDeploymentDescriptor( checkArgument(attemptNumber >= 0); this.jobID = checkNotNull(jobID); + this.jobName = checkNotNull(jobName); this.vertexID = checkNotNull(vertexID); this.executionId = checkNotNull(executionId); this.serializedExecutionConfig = checkNotNull(serializedExecutionConfig); @@ -144,6 +147,7 @@ public TaskDeploymentDescriptor( public TaskDeploymentDescriptor( JobID jobID, + String jobName, JobVertexID vertexID, ExecutionAttemptID executionId, SerializedValue serializedExecutionConfig, @@ -162,6 +166,7 @@ public TaskDeploymentDescriptor( this( jobID, + jobName, vertexID, executionId, serializedExecutionConfig, @@ -195,6 +200,8 @@ public SerializedValue getSerializedExecutionConfig() { public JobID getJobID() { return jobID; } + + public String getJobName() { return jobName; } /** * Returns the task's execution vertex ID. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index a10c463a74440..121936c82a1bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -88,6 +89,13 @@ public interface Environment { */ TaskManagerRuntimeInfo getTaskManagerInfo(); + /** + * Returns the task specific metric group. + * + * @return The MetricGroup of this task. + */ + TaskMetricGroup getMetricGroup(); + /** * Returns the job-wide configuration object that was attached to the JobGraph. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index cbc47a4aba89c..a85f32ac2eec6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -673,6 +673,7 @@ TaskDeploymentDescriptor createDeploymentDescriptor( return new TaskDeploymentDescriptor( getJobId(), + getExecutionGraph().getJobName(), getJobvertexId(), executionId, serializedConfig, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java index 7aa57a8ec339b..a784f5444ca7d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.api.reader; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; @@ -122,4 +123,11 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { deserializer.setReporter(reporter); } } + + @Override + public void setMetricGroup(IOMetricGroup metrics) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.instantiateMetrics(metrics); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java index debb352589bcf..e5f5930698785 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.reader; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; @@ -53,4 +54,8 @@ public Buffer getNextBuffer() throws IOException, InterruptedException { public void setReporter(AccumulatorRegistry.Reporter reporter) { } + + @Override + public void setMetricGroup(IOMetricGroup metrics) { + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java index a1d705f47a151..192a9abed5df5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/ReaderBase.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; @@ -57,4 +58,11 @@ public interface ReaderBase { */ void setReporter(AccumulatorRegistry.Reporter reporter); + /** + * Setter for the metric group. + * + * @param metrics metric group to set + */ + void setMetricGroup(IOMetricGroup metrics); + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java index cdd8731c3a74c..1c1747601b8fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java @@ -21,6 +21,8 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.DataInputDeserializer; @@ -47,6 +49,9 @@ public class AdaptiveSpanningRecordDeserializer im private Buffer currentBuffer; private AccumulatorRegistry.Reporter reporter; + + private transient Counter numRecordsIn; + private transient Counter numBytesIn; public AdaptiveSpanningRecordDeserializer() { this.nonSpanningWrapper = new NonSpanningWrapper(); @@ -96,6 +101,9 @@ public DeserializationResult getNextRecord(T target) throws IOException { if (reporter != null) { reporter.reportNumBytesIn(len); } + if (numBytesIn != null) { + numBytesIn.inc(len); + } if (len <= nonSpanningRemaining - 4) { // we can get a full record from here @@ -104,6 +112,9 @@ public DeserializationResult getNextRecord(T target) throws IOException { if (reporter != null) { reporter.reportNumRecordsIn(1); } + if (numRecordsIn != null) { + numRecordsIn.inc(); + } return (this.nonSpanningWrapper.remaining() == 0) ? DeserializationResult.LAST_RECORD_FROM_BUFFER : @@ -131,6 +142,9 @@ public DeserializationResult getNextRecord(T target) throws IOException { if (reporter != null) { reporter.reportNumRecordsIn(1); } + if (numRecordsIn != null) { + numRecordsIn.inc(); + } // move the remainder to the non-spanning wrapper // this does not copy it, only sets the memory segment @@ -165,6 +179,12 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { this.spanningWrapper.setReporter(reporter); } + @Override + public void instantiateMetrics(IOMetricGroup metrics) { + numBytesIn = metrics.getBytesInCounter(); + numRecordsIn = metrics.getRecordsInCounter(); + } + // ----------------------------------------------------------------------------------------------------------------- private static final class NonSpanningWrapper implements DataInputView { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java index e4c7890bd69e6..2f0c1ac65486e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordDeserializer.java @@ -23,6 +23,7 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -70,4 +71,11 @@ public boolean isBufferConsumed() { * Setter for the reporter, e.g. for the number of records emitted and the number of bytes read. */ void setReporter(AccumulatorRegistry.Reporter reporter); + + /** + * Instantiates all metrics. + * + * @param metrics metric group + */ + void instantiateMetrics(IOMetricGroup metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java index e9f339a716d97..9e7dfc1f3cf7e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -69,4 +70,11 @@ public boolean isFullBuffer() { * Setter for the reporter, e.g. for the number of records emitted and the number of bytes read. */ void setReporter(AccumulatorRegistry.Reporter reporter); + + /** + * Insantiates all metrics. + * + * @param metrics metric group + */ + void instantiateMetrics(IOMetricGroup metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java index f163e050c9fb6..64956507f7714 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializer.java @@ -24,6 +24,8 @@ import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.DataOutputSerializer; @@ -53,6 +55,9 @@ public class SpanningRecordSerializer implements R private AccumulatorRegistry.Reporter reporter; + private transient Counter numRecordsOut; + private transient Counter numBytesOut; + public SpanningRecordSerializer() { this.serializationBuffer = new DataOutputSerializer(128); @@ -85,6 +90,14 @@ public SerializationResult addRecord(T record) throws IOException { reporter.reportNumBytesOut(len); reporter.reportNumRecordsOut(1); } + + if (numBytesOut != null) { + numBytesOut.inc(len); + } + + if (numRecordsOut != null) { + numRecordsOut.inc(); + } this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer(); @@ -187,4 +200,10 @@ public boolean hasData() { public void setReporter(AccumulatorRegistry.Reporter reporter) { this.reporter = reporter; } + + @Override + public void instantiateMetrics(IOMetricGroup metrics) { + numBytesOut = metrics.getBytesOutCounter(); + numRecordsOut = metrics.getRecordsOutCounter(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java index 6a4692e7076b2..49f7584497c85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java @@ -24,6 +24,8 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.DataInputDeserializer; @@ -63,6 +65,9 @@ public class SpillingAdaptiveSpanningRecordDeserializer 0) { @@ -165,6 +176,9 @@ else if (remaining == 0) { if (reporter != null) { reporter.reportNumRecordsIn(1); } + if (numRecordsIn != null) { + numRecordsIn.inc(); + } // move the remainder to the non-spanning wrapper // this does not copy it, only sets the memory segment @@ -196,6 +210,13 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { this.spanningWrapper.setReporter(reporter); } + @Override + public void instantiateMetrics(IOMetricGroup metrics) { + numBytesIn = metrics.getBytesInCounter(); + numRecordsIn = metrics.getRecordsInCounter(); + } + + // ----------------------------------------------------------------------------------------------------------------- private static final class NonSpanningWrapper implements DataInputView { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index c534aa2188d28..f93cdfcd5343f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.api.writer; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer; @@ -209,6 +210,16 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { } } + /** + * Sets the metric group for this RecordWriter. + * @param metrics + */ + public void setMetricGroup(IOMetricGroup metrics) { + for(RecordSerializer serializer : serializers) { + serializer.instantiateMetrics(metrics); + } + } + /** * Writes the buffer to the {@link ResultPartitionWriter}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java index 434f7d4e4ed96..047fd7ea0b84f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativeTask.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.operators.BatchTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -169,10 +170,10 @@ protected void closeLocalStrategiesAndCaches() { } @Override - public DistributedRuntimeUDFContext createRuntimeContext() { + public DistributedRuntimeUDFContext createRuntimeContext(MetricGroup metrics) { Environment env = getEnvironment(); return new IterativeRuntimeUdfContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap); + getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap, metrics); } // -------------------------------------------------------------------------------------------- @@ -362,8 +363,8 @@ private TypeSerializer getOutputSerializer() { private class IterativeRuntimeUdfContext extends DistributedRuntimeUDFContext implements IterationRuntimeContext { public IterativeRuntimeUdfContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulatorMap) { - super(taskInfo, userCodeClassLoader, executionConfig, cpTasks, accumulatorMap); + Map> cpTasks, Map> accumulatorMap, MetricGroup metrics) { + super(taskInfo, userCodeClassLoader, executionConfig, cpTasks, accumulatorMap, metrics); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java index 51e00b38597b5..546193c4892d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableMaterialization; import org.apache.flink.runtime.execution.CancelTaskException; @@ -212,6 +213,7 @@ public class BatchTask extends AbstractInvokable impleme * The accumulator map used in the RuntimeContext. */ protected Map> accumulatorMap; + private MetricGroup metrics; // -------------------------------------------------------------------------------------------- // Task Interface @@ -237,6 +239,10 @@ public void invoke() throws Exception { final Class> driverClass = this.config.getDriver(); this.driver = InstantiationUtil.instantiate(driverClass, Driver.class); + String headName = getEnvironment().getTaskInfo().getTaskName().split("->")[0].trim(); + this.metrics = getEnvironment().getMetricGroup() + .addOperator(headName.startsWith("CHAIN") ? headName.substring(6) : headName); + // initialize the readers. // this does not yet trigger any stream consuming or processing. initInputReaders(); @@ -256,7 +262,7 @@ public void invoke() throws Exception { LOG.debug(formatLogString("Start task code.")); } - this.runtimeUdfContext = createRuntimeContext(); + this.runtimeUdfContext = createRuntimeContext(metrics); // whatever happens in this scope, make sure that the local strategies are cleaned up! // note that the initialization of the local strategies is in the try-finally block as well, @@ -666,6 +672,7 @@ protected void initInputReaders() throws Exception { } inputReaders[i].setReporter(reporter); + inputReaders[i].setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); currentReaderOffset += groupSize; } @@ -1005,11 +1012,11 @@ protected void initOutputs() throws Exception { this.getExecutionConfig(), reporter, this.accumulatorMap); } - public DistributedRuntimeUDFContext createRuntimeContext() { + public DistributedRuntimeUDFContext createRuntimeContext(MetricGroup metrics) { Environment env = getEnvironment(); return new DistributedRuntimeUDFContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap); + getExecutionConfig(), env.getDistributedCacheEntries(), this.accumulatorMap, metrics); } // -------------------------------------------------------------------------------------------- @@ -1056,6 +1063,11 @@ public String formatLogString(String message) { return constructLogString(message, getEnvironment().getTaskInfo().getTaskName(), this); } + @Override + public MetricGroup getMetricGroup() { + return metrics; + } + @Override public MutableObjectIterator getInput(int index) { if (index < 0 || index > this.driver.getNumberOfInputs()) { @@ -1226,6 +1238,7 @@ public static Collector getOutputCollector(AbstractInvokable task, TaskCo // setup live accumulator counters recordWriter.setReporter(reporter); + recordWriter.setMetricGroup(task.getEnvironment().getMetricGroup().getIOMetricGroup()); writers.add(recordWriter); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index 21e87843bbd74..39bf23f126167 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -344,6 +344,8 @@ private void initInputReaders() throws Exception { final AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter(); inputReader.setReporter(reporter); + + inputReader.setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); this.inputTypeSerializerFactory = this.config.getInputSerializer(0, getUserCodeClassLoader()); @SuppressWarnings({ "rawtypes" }) @@ -375,6 +377,7 @@ public DistributedRuntimeUDFContext createRuntimeContext() { Environment env = getEnvironment(); return new DistributedRuntimeUDFContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap()); + getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap(), + getEnvironment().getMetricGroup().addOperator(getEnvironment().getTaskInfo().getTaskName())); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 0c525ea80978e..819b84f99dd5f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -358,7 +358,10 @@ public void remove() { public DistributedRuntimeUDFContext createRuntimeContext() { Environment env = getEnvironment(); + String sourceName = getEnvironment().getTaskInfo().getTaskName().split("->")[0].trim(); + sourceName = sourceName.startsWith("CHAIN") ? sourceName.substring(6) : sourceName; return new DistributedRuntimeUDFContext(env.getTaskInfo(), getUserCodeClassLoader(), - getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap()); + getExecutionConfig(), env.getDistributedCacheEntries(), env.getAccumulatorRegistry().getUserMap(), + getEnvironment().getMetricGroup().addOperator(sourceName)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java index 220527d6f8d58..d6825acb102be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java @@ -134,4 +134,4 @@ public void cleanup() {} public void cancel() { this.running = false; } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java index fd5d2385d2338..df225288ac407 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TaskContext.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; @@ -67,4 +68,6 @@ public interface TaskContext { AbstractInvokable getOwningNepheleTask(); String formatLogString(String message); + + MetricGroup getMetricGroup(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java index 2fb52cde980e2..407716f92f453 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.BatchTask; @@ -50,6 +51,8 @@ public abstract class ChainedDriver implements Collector { protected ExecutionConfig executionConfig; protected boolean objectReuseEnabled = false; + + protected MetricGroup metrics; public void setup(TaskConfig config, String taskName, Collector outputCollector, @@ -60,14 +63,15 @@ public void setup(TaskConfig config, String taskName, Collector outputCollec this.taskName = taskName; this.outputCollector = outputCollector; this.userCodeClassLoader = userCodeClassLoader; + this.metrics = parent.getEnvironment().getMetricGroup().addOperator(taskName); Environment env = parent.getEnvironment(); if (parent instanceof BatchTask) { - this.udfContext = ((BatchTask) parent).createRuntimeContext(); + this.udfContext = ((BatchTask) parent).createRuntimeContext(metrics); } else { this.udfContext = new DistributedRuntimeUDFContext(env.getTaskInfo(), userCodeClassLoader, - parent.getExecutionConfig(), env.getDistributedCacheEntries(), accumulatorMap + parent.getExecutionConfig(), env.getDistributedCacheEntries(), accumulatorMap, metrics ); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java index b5ac4d788ad5f..293d34f44a283 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.broadcast.BroadcastVariableMaterialization; import org.apache.flink.runtime.broadcast.InitializationTypeConflictException; @@ -43,8 +44,8 @@ public class DistributedRuntimeUDFContext extends AbstractRuntimeUDFContext { private final HashMap> broadcastVars = new HashMap>(); public DistributedRuntimeUDFContext(TaskInfo taskInfo, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, - Map> cpTasks, Map> accumulators) { - super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks); + Map> cpTasks, Map> accumulators, MetricGroup metrics) { + super(taskInfo, userCodeClassLoader, executionConfig, accumulators, cpTasks, metrics); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 51e7e34b3064f..1f93a0d9bfc80 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; @@ -75,6 +76,7 @@ public class RuntimeEnvironment implements Environment { private final AccumulatorRegistry accumulatorRegistry; private final TaskManagerRuntimeInfo taskManagerInfo; + private final TaskMetricGroup metrics; // ------------------------------------------------------------------------ @@ -96,7 +98,8 @@ public RuntimeEnvironment( ResultPartitionWriter[] writers, InputGate[] inputGates, ActorGateway jobManager, - TaskManagerRuntimeInfo taskManagerInfo) { + TaskManagerRuntimeInfo taskManagerInfo, + TaskMetricGroup metrics) { this.jobId = checkNotNull(jobId); this.jobVertexId = checkNotNull(jobVertexId); @@ -116,6 +119,7 @@ public RuntimeEnvironment( this.inputGates = checkNotNull(inputGates); this.jobManager = checkNotNull(jobManager); this.taskManagerInfo = checkNotNull(taskManagerInfo); + this.metrics = metrics; } // ------------------------------------------------------------------------ @@ -160,6 +164,11 @@ public TaskManagerRuntimeInfo getTaskManagerInfo() { return taskManagerInfo; } + @Override + public TaskMetricGroup getMetricGroup() { + return metrics; + } + @Override public ClassLoader getUserClassLoader() { return userCodeClassLoader; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 251673fcf6288..bc3ba0d9bc906 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -201,6 +202,8 @@ public class Task implements Runnable { /** atomic flag that makes sure the invokable is canceled exactly once upon error */ private final AtomicBoolean invokableHasBeenCanceled; + private final TaskMetricGroup metrics; + /** The invokable of this task, if initialized */ private volatile AbstractInvokable invokable; @@ -239,7 +242,8 @@ public Task(TaskDeploymentDescriptor tdd, FiniteDuration actorAskTimeout, LibraryCacheManager libraryCache, FileCache fileCache, - TaskManagerRuntimeInfo taskManagerConfig) + TaskManagerRuntimeInfo taskManagerConfig, + TaskMetricGroup metricGroup) { this.taskInfo = checkNotNull(tdd.getTaskInfo()); this.jobId = checkNotNull(tdd.getJobID()); @@ -274,6 +278,7 @@ public Task(TaskDeploymentDescriptor tdd, this.taskManagerConfig = checkNotNull(taskManagerConfig); this.executionListenerActors = new CopyOnWriteArrayList(); + this.metrics = metricGroup; // create the reader and writer structures @@ -518,7 +523,7 @@ else if (current == ExecutionState.CANCELING) { userCodeClassLoader, memoryManager, ioManager, broadcastVariableManager, accumulatorRegistry, splitProvider, distributedCacheEntries, - writers, inputGates, jobManager, taskManagerConfig); + writers, inputGates, jobManager, taskManagerConfig, metrics); // let the task code create its readers and writers invokable.setEnvironment(env); @@ -683,6 +688,9 @@ else if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) { // remove all of the tasks library resources libraryCache.unregisterTask(jobId, executionId); + + //Uncomment before Merging!!! + //metrics.close(); // remove all files in the distributed cache removeCachedFiles(distributedCacheEntries, fileCache); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 1c7815c17984b..beb012c9bed1f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -18,13 +18,14 @@ package org.apache.flink.runtime.taskmanager -import java.io.{FileInputStream, File, IOException} +import java.io.{File, FileInputStream, IOException} import java.lang.management.{ManagementFactory, OperatingSystemMXBean} import java.lang.reflect.Method import java.net.{InetAddress, InetSocketAddress} import java.util import java.util.UUID import java.util.concurrent.TimeUnit +import javax.management.ObjectName import _root_.akka.actor._ import _root_.akka.pattern.ask @@ -36,12 +37,14 @@ import com.fasterxml.jackson.databind.ObjectMapper import grizzled.slf4j.Logger import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem -import org.apache.flink.core.memory.{HybridMemorySegment, HeapMemorySegment, MemorySegmentFactory, MemoryType} +import org.apache.flink.core.memory.{HeapMemorySegment, HybridMemorySegment, MemorySegmentFactory, MemoryType} +import org.apache.flink.metrics.groups.TaskManagerMetricGroup +import org.apache.flink.metrics.{MetricGroup, Gauge => FlinkGauge, MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.blob.{BlobClient, BlobCache, BlobService} +import org.apache.flink.runtime.blob.{BlobCache, BlobClient, BlobService} import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, TaskDeploymentDescriptor} import org.apache.flink.runtime.execution.ExecutionState @@ -58,7 +61,7 @@ import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, Leader import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.messages.Messages._ import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.StackTraceSampleMessages.{SampleTaskStackTrace, ResponseStackTraceSampleFailure, ResponseStackTraceSampleSuccess, StackTraceSampleMessages, TriggerStackTraceSample} +import org.apache.flink.runtime.messages.StackTraceSampleMessages.{ResponseStackTraceSampleFailure, ResponseStackTraceSampleSuccess, SampleTaskStackTrace, StackTraceSampleMessages, TriggerStackTraceSample} import org.apache.flink.runtime.messages.TaskManagerMessages._ import org.apache.flink.runtime.messages.TaskMessages._ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, NotifyCheckpointComplete, TriggerCheckpoint} @@ -152,6 +155,9 @@ class TaskManager( /** Registry of metrics periodically transmitted to the JobManager */ private val metricRegistry = TaskManager.createMetricsRegistry() + private var metricsRegistry : FlinkMetricRegistry = null + private var taskManagerMetricGroup : TaskManagerMetricGroup = null + /** Metric serialization */ private val metricRegistryMapper: ObjectMapper = new ObjectMapper() .registerModule( @@ -257,6 +263,13 @@ class TaskManager( } catch { case t: Exception => log.error("FileCache did not shutdown properly.", t) } + + try { + //enable this before merging + //metricsRegistry.shutdown() + } catch { + case t: Exception => log.error("MetricRegistry did not shutdown properly.", t) + } log.info(s"Task manager ${self.path} is completely shut down.") } @@ -924,6 +937,14 @@ class TaskManager( else { libraryCacheManager = Some(new FallbackLibraryCacheManager) } + + metricsRegistry = new FlinkMetricRegistry(this.config.configuration); + + taskManagerMetricGroup = + new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString) + + TaskManager.instantiateStatusMetrics(taskManagerMetricGroup) + // watch job manager to detect when it dies context.watch(jobManager) @@ -1056,6 +1077,17 @@ class TaskManager( val jobManagerGateway = new AkkaActorGateway(jobManagerActor, leaderSessionID.orNull) val selfGateway = new AkkaActorGateway(self, leaderSessionID.orNull) + var jobName = tdd.getJobName + if (tdd.getJobName.length == 0) { + jobName = tdd.getJobID.toString() + } else { + jobName = tdd.getJobName + } + + val taskMetricGroup = taskManagerMetricGroup + .addJob(tdd.getJobID, jobName) + .addTask(tdd.getVertexID, tdd.getExecutionId, tdd.getIndexInSubtaskGroup, tdd.getTaskName) + val task = new Task( tdd, memoryManager, @@ -1067,7 +1099,8 @@ class TaskManager( config.timeout, libCache, fileCache, - runtimeInfo) + runtimeInfo, + taskMetricGroup) log.info(s"Received task ${task.getTaskInfo.getTaskNameWithSubtasks()}") @@ -2234,4 +2267,130 @@ object TaskManager { }) metricRegistry } + + private def instantiateStatusMetrics(taskManagerMetricGroup: MetricGroup) : Unit = { + val jvm = taskManagerMetricGroup + .addGroup("Status") + .addGroup("JVM") + + instantiateClassLoaderMetrics(jvm.addGroup("ClassLoader")) + instantiateGarbageCollectorMetrics(jvm.addGroup("GarbageCollector")) + instantiateMemoryMetrics(jvm.addGroup("Memory")) + instantiateThreadMetrics(jvm.addGroup("Threads")) + instantiateCPUMetrics(jvm.addGroup("CPU")) + } + + private def instantiateClassLoaderMetrics(metrics: MetricGroup) { + val mxBean = ManagementFactory.getClassLoadingMXBean + + metrics + .gauge("ClassesLoaded", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getTotalLoadedClassCount + }) + metrics.gauge("ClassesUnloaded", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getUnloadedClassCount + }) + } + + private def instantiateGarbageCollectorMetrics(metrics: MetricGroup) { + val garbageCollectors = ManagementFactory.getGarbageCollectorMXBeans + + for (garbageCollector <- garbageCollectors) { + val gcGroup = metrics.addGroup("\"" + garbageCollector.getName + "\"") + gcGroup.gauge("Count", new FlinkGauge[Long] { + override def getValue: Long = garbageCollector.getCollectionCount + }) + gcGroup.gauge("Time", new FlinkGauge[Long] { + override def getValue: Long = garbageCollector.getCollectionTime + }) + } + } + + private def instantiateMemoryMetrics(metrics: MetricGroup) { + val mxBean = ManagementFactory.getMemoryMXBean + val heap = metrics.addGroup("Heap") + heap.gauge("Used", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getHeapMemoryUsage.getUsed + }) + heap.gauge("Committed", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getHeapMemoryUsage.getCommitted + }) + heap.gauge("Max", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getHeapMemoryUsage.getMax + }) + + val nonHeap = metrics.addGroup("NonHeap") + nonHeap.gauge("Used", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getNonHeapMemoryUsage.getUsed + }) + nonHeap.gauge("Committed", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getNonHeapMemoryUsage.getCommitted + }) + nonHeap.gauge("Max", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getNonHeapMemoryUsage.getMax + }) + + val con = ManagementFactory.getPlatformMBeanServer; + + val directObjectName = new ObjectName("java.nio:type=BufferPool,name=direct") + + val direct = metrics.addGroup("Direct") + direct.gauge("Count", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(directObjectName, "Count").asInstanceOf[Long] + }) + direct.gauge("MemoryUsed", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(directObjectName, "MemoryUsed").asInstanceOf[Long] + }) + direct.gauge("TotalCapacity", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(directObjectName, "TotalCapacity").asInstanceOf[Long] + }) + + val mappedObjectName = new ObjectName("java.nio:type=BufferPool,name=direct") + + val mapped = metrics.addGroup("Mapped") + mapped.gauge("Count", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(mappedObjectName, "Count").asInstanceOf[Long] + }) + mapped.gauge("MemoryUsed", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(mappedObjectName, "MemoryUsed").asInstanceOf[Long] + }) + mapped.gauge("TotalCapacity", new FlinkGauge[Long] { + override def getValue: Long = con + .getAttribute(mappedObjectName, "TotalCapacity").asInstanceOf[Long] + }) + } + + private def instantiateThreadMetrics(metrics: MetricGroup): Unit = { + val mxBean = ManagementFactory.getThreadMXBean + + metrics + .gauge("Count", new FlinkGauge[Int] { + override def getValue: Int = mxBean.getThreadCount + }) + } + + private def instantiateCPUMetrics(metrics: MetricGroup): Unit = { + try { + val mxBean = ManagementFactory.getOperatingSystemMXBean + .asInstanceOf[com.sun.management.OperatingSystemMXBean] + + metrics + .gauge("Load", new FlinkGauge[Double] { + override def getValue: Double = mxBean.getProcessCpuLoad + }) + metrics.gauge("Time", new FlinkGauge[Long] { + override def getValue: Long = mxBean.getProcessCpuTime + }) + } + catch { + case t: Throwable => + LOG.warn("Cannot access com.sun.management.OperatingSystemMXBean.getProcessCpuLoad()" + + " - CPU load metrics will not be available.") + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 36744a91815f6..f0bf69428c47b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -46,6 +46,7 @@ public void testSerialization() { final JobID jobID = new JobID(); final JobVertexID vertexID = new JobVertexID(); final ExecutionAttemptID execId = new ExecutionAttemptID(); + final String jobName = "job name"; final String taskName = "task name"; final int indexInSubtaskGroup = 0; final int currentNumberOfSubtasks = 1; @@ -59,7 +60,7 @@ public void testSerialization() { final List requiredClasspaths = new ArrayList(0); final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); - final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, + final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, jobName, vertexID, execId, executionConfig, taskName, indexInSubtaskGroup, currentNumberOfSubtasks, attemptNumber, jobConfiguration, taskConfiguration, invokableClass.getName(), producedResults, inputGates, requiredJars, requiredClasspaths, 47); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java index 6853722330c7a..9724a809c628e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java @@ -20,6 +20,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; @@ -189,5 +190,9 @@ protected MockReader(InputGate inputGate) { public void setReporter(AccumulatorRegistry.Reporter reporter) { } + + @Override + public void setMetricGroup(IOMetricGroup metrics) { + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index 9be957aa8077d..0300a079b1f39 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -25,6 +25,8 @@ import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; @@ -221,4 +223,9 @@ public AbstractInvokable getOwningNepheleTask() { public String formatLogString(String message) { return message; } + + @Override + public MetricGroup getMetricGroup() { + return new DummyMetricGroup(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java index 7043a637b0c42..2c3dcf144ede2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java @@ -26,6 +26,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -366,6 +368,11 @@ public String formatLogString(String message) { return "Driver Tester: " + message; } + @Override + public MetricGroup getMetricGroup() { + return new DummyMetricGroup(); + } + // -------------------------------------------------------------------------------------------- @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index ab58cea536d98..638173359c8db 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -24,6 +24,8 @@ import java.util.List; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.util.DummyMetricGroup; import org.apache.flink.runtime.operators.Driver; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.TestLogger; @@ -363,6 +365,11 @@ public String formatLogString(String message) { return "Driver Tester: " + message; } + @Override + public MetricGroup getMetricGroup() { + return new DummyMetricGroup(); + } + // -------------------------------------------------------------------------------------------- @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java index 233dace253e9b..9b54383e5ca1f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java @@ -27,6 +27,8 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; @@ -81,6 +83,11 @@ public TaskManagerRuntimeInfo getTaskManagerInfo() { return null; } + @Override + public TaskMetricGroup getMetricGroup() { + return new DummyTaskMetricGroup(); + } + @Override public Configuration getJobConfiguration() { return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index d29b20670617c..31fd08c2957f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -24,6 +24,8 @@ import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.metrics.groups.TaskMetricGroup; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; @@ -209,6 +211,11 @@ public TaskManagerRuntimeInfo getTaskManagerInfo() { return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); } + @Override + public TaskMetricGroup getMetricGroup() { + return new DummyTaskMetricGroup(); + } + @Override public InputSplitProvider getInputSplitProvider() { return this.inputSplitProvider; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 886c881bf6734..77e18c60eac2a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -358,6 +359,11 @@ public String formatLogString(String message) { return "Driver Tester: " + message; } + @Override + public MetricGroup getMetricGroup() { + return null; + } + // -------------------------------------------------------------------------------------------- @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 1b463bcd38702..ae05ae9e7a9cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -148,7 +149,7 @@ private static Task createTask() { when(networkEnvironment.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new JobID(), "Job Name", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), "Test Task", 0, 1, 0, new Configuration(), new Configuration(), @@ -170,7 +171,8 @@ private static Task createTask() { new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration())); + new TaskManagerRuntimeInfo("localhost", new Configuration()), + mock(TaskMetricGroup.class)); } public static class CheckpointsInOrderInvokable extends AbstractInvokable implements StatefulTask> { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 3ee9a842aa39e..b3ad5894e9206 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -162,7 +162,7 @@ protected void run() { final ExecutionAttemptID eid = new ExecutionAttemptID(); final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); - final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, executionConfig, + final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, "TestJob", vid, eid, executionConfig, "TestTask", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(), Collections.emptyList(), @@ -263,14 +263,14 @@ public void testJobSubmissionAndCanceling() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, "TestJob1", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, "TestJob2", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), @@ -399,13 +399,13 @@ public void testJobSubmissionAndStop() { final SerializedValue executionConfig = ExecutionConfigTest.getSerializedConfig(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, executionConfig, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, "TestJob", vid1, eid1, executionConfig, "TestTask1", 1, 5, 0, new Configuration(), new Configuration(), StoppableInvokable.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, executionConfig, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, "TestJob", vid2, eid2, executionConfig, "TestTask2", 2, 7, 0, new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(), Collections.emptyList(), Collections.emptyList(), @@ -525,14 +525,14 @@ public void testGateChannelEdgeMismatch() { final ExecutionAttemptID eid1 = new ExecutionAttemptID(); final ExecutionAttemptID eid2 = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, "TestJob", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), Collections.emptyList(), Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, "TestJob", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), @@ -626,13 +626,13 @@ public void testRunJobWithForwardChannel() { } ); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, "TestJob", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, "TestJob", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.Receiver.class.getName(), Collections.emptyList(), @@ -767,13 +767,13 @@ public void testCancellingDependentAndStateUpdateFails() { } ); - final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, + final TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, "TestJob", vid1, eid1, ExecutionConfigTest.getSerializedConfig(), "Sender", 0, 1, 0, new Configuration(), new Configuration(), Tasks.Sender.class.getName(), irpdd, Collections.emptyList(), new ArrayList(), Collections.emptyList(), 0); - final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, + final TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, "TestJob", vid2, eid2, ExecutionConfigTest.getSerializedConfig(), "Receiver", 2, 7, 0, new Configuration(), new Configuration(), Tasks.BlockingReceiver.class.getName(), Collections.emptyList(), @@ -912,7 +912,7 @@ public void testRemotePartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, + jid, "TestJob", vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), @@ -1007,7 +1007,7 @@ public void testLocalPartitionNotFound() throws Exception { new InputGateDeploymentDescriptor(resultId, 0, icdd); final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - jid, vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, + jid, "TestJob", vid, eid, ExecutionConfigTest.getSerializedConfig(), "Receiver", 0, 1, 0, new Configuration(), new Configuration(), Tasks.AgnosticReceiver.class.getName(), Collections.emptyList(), @@ -1081,6 +1081,7 @@ public void testTriggerStackTraceSampleMessage() throws Exception { // Single blocking task final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), + "Job", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index 99e037d452b00..7660893ab2969 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; @@ -69,7 +70,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { task = new Task(tddMock, mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), mock(BroadcastVariableManager.class), mock(ActorGateway.class), mock(ActorGateway.class), mock(FiniteDuration.class), mock(LibraryCacheManager.class), mock(FileCache.class), - mock(TaskManagerRuntimeInfo.class)); + mock(TaskManagerRuntimeInfo.class), mock(TaskMetricGroup.class)); Field f = task.getClass().getDeclaredField("invokable"); f.setAccessible(true); f.set(task, taskMock); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 06f393f30c4a9..f237c877aaea9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -622,12 +623,13 @@ private Task createTask(Class invokable, new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration())); + new TaskManagerRuntimeInfo("localhost", new Configuration()), + mock(TaskMetricGroup.class)); } private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { return new TaskDeploymentDescriptor( - new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new JobID(), "Test Job", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), "Test Task", 0, 1, 0, new Configuration(), new Configuration(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index c9b1d38de8be1..783b3e27af5ba 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -53,6 +53,7 @@ public class StreamConfig implements Serializable { private static final String CHAINED_OUTPUTS = "chainedOutputs"; private static final String CHAINED_TASK_CONFIG = "chainedTaskConfig_"; private static final String IS_CHAINED_VERTEX = "isChainedSubtask"; + private static final String CHAIN_INDEX = "chainIndex"; private static final String VERTEX_NAME = "vertexID"; private static final String ITERATION_ID = "iterationId"; private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper"; @@ -387,6 +388,14 @@ public Map getTransitiveChainedTaskConfigs(ClassLoader cl throw new StreamTaskException("Could not instantiate configuration.", e); } } + + public void setChainIndex(int index) { + this.config.setInteger(CHAIN_INDEX, index); + } + + public int getChainIndex() { + return this.config.getInteger(CHAIN_INDEX, 0); + } // ------------------------------------------------------------------------ // State backend diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 9adabae33ec04..71cc7f2010f9b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -167,14 +167,15 @@ private void setPhysicalEdges() { */ private void setChaining(Map hashes) { for (Integer sourceNodeId : streamGraph.getSourceIDs()) { - createChain(sourceNodeId, sourceNodeId, hashes); + createChain(sourceNodeId, sourceNodeId, hashes, 0); } } private List createChain( Integer startNodeId, Integer currentNodeId, - Map hashes) { + Map hashes, + int chainIndex) { if (!builtVertices.contains(startNodeId)) { @@ -192,12 +193,12 @@ private List createChain( } for (StreamEdge chainable : chainableOutputs) { - transitiveOutEdges.addAll(createChain(startNodeId, chainable.getTargetId(), hashes)); + transitiveOutEdges.addAll(createChain(startNodeId, chainable.getTargetId(), hashes, chainIndex + 1)); } for (StreamEdge nonChainable : nonChainableOutputs) { transitiveOutEdges.add(nonChainable); - createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes); + createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes, 0); } chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs)); @@ -211,6 +212,7 @@ private List createChain( if (currentNodeId.equals(startNodeId)) { config.setChainStart(); + config.setChainIndex(0); config.setOutEdgesInOrder(transitiveOutEdges); config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges()); @@ -227,6 +229,7 @@ private List createChain( if (chainedConfs == null) { chainedConfigs.put(startNodeId, new HashMap()); } + config.setChainIndex(chainIndex); chainedConfigs.get(startNodeId).put(currentNodeId, config); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 9673f874e80f0..326a42f619f2e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.runtime.state.KvStateSnapshot; import org.apache.flink.runtime.state.AbstractStateBackend; @@ -91,6 +92,7 @@ public abstract class AbstractStreamOperator /** The state backend that stores the state and checkpoints for this task */ private AbstractStateBackend stateBackend = null; + protected MetricGroup metrics; // ------------------------------------------------------------------------ // Life Cycle @@ -101,6 +103,9 @@ public void setup(StreamTask containingTask, StreamConfig config, Output")[config.getChainIndex()].trim(); + + this.metrics = container.getEnvironment().getMetricGroup().addOperator(operatorName); this.runtimeContext = new StreamingRuntimeContext(this, container.getEnvironment(), container.getAccumulatorMap()); stateKeySelector1 = config.getStatePartitioner(0, getUserCodeClassloader()); @@ -116,6 +121,10 @@ public void setup(StreamTask containingTask, StreamConfig config, Output extends Serializable { ChainingStrategy getChainingStrategy(); void setChainingStrategy(ChainingStrategy strategy); + + MetricGroup getMetricGroup(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java index 4500ee73a1161..5fef3c797dbd5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java @@ -64,7 +64,8 @@ public StreamingRuntimeContext(AbstractStreamOperator operator, env.getUserClassLoader(), operator.getExecutionConfig(), accumulators, - env.getDistributedCacheEntries()); + env.getDistributedCacheEntries(), + operator.getMetricGroup()); this.operator = operator; this.taskEnvironment = env; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 8c0b63f8eef88..1dde85b3c9f10 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -202,6 +203,17 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { deserializer.setReporter(reporter); } } + + /** + * Sets the metric group for this StreamInputProcessor. + * + * @param metrics metric group + */ + public void setMetricGroup(IOMetricGroup metrics) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.instantiateMetrics(metrics); + } + } public void cleanup() throws IOException { // clear the buffers first. this part should not ever fail diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 4adea3bf8ab0a..07ada23502ccf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.groups.IOMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -278,6 +279,17 @@ public void setReporter(AccumulatorRegistry.Reporter reporter) { deserializer.setReporter(reporter); } } + + /** + * Sets the metric group for this StreamTwoInputProcessor. + * + * @param metrics metric group + */ + public void setMetricGroup(IOMetricGroup metrics) { + for (RecordDeserializer deserializer : recordDeserializers) { + deserializer.instantiateMetrics(metrics); + } + } public void cleanup() throws IOException { // clear the buffers first. this part should not ever fail diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index a1ed8288c26e7..938d8c147bcc5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -52,6 +52,7 @@ public void init() throws Exception { AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); inputProcessor.setReporter(reporter); + inputProcessor.setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index f3d3482ce692c..90abea40a5600 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -286,6 +286,7 @@ private static RecordWriterOutput createStreamOutput( StreamRecordWriter>> output = new StreamRecordWriter<>(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout()); output.setReporter(reporter); + output.setMetricGroup(taskEnvironment.getMetricGroup().getIOMetricGroup()); return new RecordWriterOutput(output, outSerializer, withTimestamps); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index ebc7789c04852..c3305eb6e2d07 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -77,6 +77,7 @@ public void init() throws Exception { AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry(); AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter(); this.inputProcessor.setReporter(reporter); + inputProcessor.setMetricGroup(getEnvironment().getMetricGroup().getIOMetricGroup()); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java index dff118424631a..812507f509cff 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java @@ -29,6 +29,7 @@ import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; @@ -794,6 +795,7 @@ public void apply(Integer key, final Environment env = mock(Environment.class); when(env.getTaskInfo()).thenReturn(new TaskInfo("Test task name", 0, 1, 0)); when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader()); + when(env.getMetricGroup()).thenReturn(new DummyTaskMetricGroup()); when(task.getEnvironment()).thenReturn(env); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java index 342921510e5a2..c89ac5080e609 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java @@ -34,6 +34,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.util.DummyTaskMetricGroup; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.Output; @@ -966,6 +967,7 @@ public Tuple2 reduce(Tuple2 value1, Tuple2 state) { public TaskManagerRuntimeInfo getTaskManagerInfo() { return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); } + + @Override + public TaskMetricGroup getMetricGroup() { + return new DummyTaskMetricGroup(); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index ed8bf019f7e02..deda82fde386c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.ExecutionConfigTest; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -134,7 +135,7 @@ private Task createTask(Class invokable, StreamConf when(network.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( - new JobID(), new JobVertexID(), new ExecutionAttemptID(), + new JobID(), "Job Name", new JobVertexID(), new ExecutionAttemptID(), ExecutionConfigTest.getSerializedConfig(), "Test Task", 0, 1, 0, new Configuration(), @@ -157,7 +158,8 @@ private Task createTask(Class invokable, StreamConf new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration())); + new TaskManagerRuntimeInfo("localhost", new Configuration()), + mock(TaskMetricGroup.class)); } // ------------------------------------------------------------------------ diff --git a/pom.xml b/pom.xml index 1ba237f596e9b..c97815af63576 100644 --- a/pom.xml +++ b/pom.xml @@ -74,6 +74,7 @@ under the License. flink-quickstart flink-contrib flink-dist + flink-metrics @@ -103,6 +104,7 @@ under the License. 3.4.6 2.8.0 2.7.4 + 3.1.0 From bcd6ae032784ed988ef23cb55b615f2653379905 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 20 May 2016 09:49:08 +0200 Subject: [PATCH 47/70] [FLINK-1502] [core] Various improvements to the metrics infrastructure code - Metric groups are generally thread-safe - Metric groups are closable. Closed groups do not register metric objects and more. - TaskManager's JobMetricsGroup auto disposes when all TaskMetricGroups are closed - Maven project with metric reporters renamed to 'flink-metric-reporters' - Various code style cleanups --- .../common/operators/CollectionExecutor.java | 21 +- .../org/apache/flink/metrics/Counter.java | 7 +- .../java/org/apache/flink/metrics/Gauge.java | 3 +- .../org/apache/flink/metrics/MetricGroup.java | 50 ++-- .../apache/flink/metrics/MetricRegistry.java | 219 ++++++++++++------ .../metrics/groups/AbstractMetricGroup.java | 179 ++++++++++---- .../metrics/groups/ComponentMetricGroup.java | 56 ++++- .../metrics/groups/GenericMetricGroup.java | 3 +- .../flink/metrics/groups/IOMetricGroup.java | 9 +- .../flink/metrics/groups/JobMetricGroup.java | 78 +++++-- .../groups/NonRegisteringMetricsGroup.java | 87 +++++++ .../metrics/groups/OperatorMetricGroup.java | 13 +- .../apache/flink/metrics/groups/Scope.java | 5 +- .../groups/TaskManagerMetricGroup.java | 96 ++++++-- .../flink/metrics/groups/TaskMetricGroup.java | 78 +++++-- .../metrics/reporter/AbstractReporter.java | 6 +- .../flink/metrics/reporter/JMXReporter.java | 7 +- .../metrics/reporter/MetricReporter.java | 1 + .../flink/metrics/groups/JobGroupTest.java | 25 +- .../MetricGroupRegistrationTest.java} | 33 +-- .../flink/metrics/groups/MetricGroupTest.java | 125 ++++++++++ .../metrics/groups/OperatorGroupTest.java | 13 +- .../flink/metrics/groups/TaskGroupTest.java | 15 +- .../metrics/groups/TaskManagerGroupTest.java | 99 +++++++- .../metrics/util/DummyJobMetricGroup.java | 9 +- .../flink/metrics/util/DummyMetricGroup.java | 18 +- .../metrics/util/DummyMetricRegistry.java | 7 +- .../util/DummyOperatorMetricGroup.java | 10 +- .../util/DummyTaskManagerMetricGroup.java | 10 +- .../metrics/util/DummyTaskMetricGroup.java | 10 +- .../flink/metrics/util/TestReporter.java | 10 +- .../flink-metrics-dropwizard/pom.xml | 2 +- .../ScheduledDropwizardReporter.java | 2 +- .../dropwizard/metrics/CounterWrapper.java | 0 .../dropwizard/metrics/GaugeWrapper.java | 8 +- .../flink-metrics-ganglia/pom.xml | 2 +- .../metrics/graphite/GangliaReporter.java | 0 .../flink-metrics-graphite/pom.xml | 2 +- .../metrics/graphite/GraphiteReporter.java | 0 .../flink-metrics-statsd/pom.xml | 2 +- .../flink/metrics/statsd/StatsDReporter.java | 2 +- .../pom.xml | 4 +- .../flink/runtime/taskmanager/Task.java | 34 ++- .../runtime/taskmanager/TaskManager.scala | 33 +-- pom.xml | 2 +- 45 files changed, 1051 insertions(+), 344 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/metrics/groups/NonRegisteringMetricsGroup.java rename flink-core/src/test/java/org/apache/flink/metrics/{MetricGroupTest.java => groups/MetricGroupRegistrationTest.java} (77%) create mode 100644 flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupTest.java rename {flink-metrics => flink-metric-reporters}/flink-metrics-dropwizard/pom.xml (97%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java (97%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java (100%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java (85%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-ganglia/pom.xml (98%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java (100%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-graphite/pom.xml (97%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java (100%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-statsd/pom.xml (96%) rename {flink-metrics => flink-metric-reporters}/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java (98%) rename {flink-metrics => flink-metric-reporters}/pom.xml (94%) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java index 7e5269e84fe87..97f0c5e500601 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java @@ -59,15 +59,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.MetricRegistry; -import org.apache.flink.metrics.groups.JobMetricGroup; -import org.apache.flink.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.metrics.groups.NonRegisteringMetricsGroup; import org.apache.flink.types.Value; -import org.apache.flink.util.AbstractID; import org.apache.flink.util.Visitor; /** @@ -93,8 +89,6 @@ public class CollectionExecutor { private final ExecutionConfig executionConfig; private int iterationSuperstep; - - private JobMetricGroup jobMetricGroup; // -------------------------------------------------------------------------------------------- @@ -120,9 +114,7 @@ public JobExecutionResult execute(Plan program) throws Exception { if (jobID == null) { jobID = new JobID(); } - this.jobMetricGroup = - new TaskManagerMetricGroup(new MetricRegistry(new Configuration()), "localhost", new AbstractID().toString()) - .addJob(jobID, program.getJobName()); + initCache(program.getCachedFiles()); Collection> sinks = program.getDataSinks(); for (Operator sink : sinks) { @@ -202,7 +194,7 @@ private void executeDataSink(GenericDataSinkBase sink, int superStep) th TaskInfo taskInfo = new TaskInfo(typedSink.getName(), 0, 1, 0); RuntimeUDFContext ctx; - MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, typedSink.getName()); + MetricGroup metrics = NonRegisteringMetricsGroup.get(); if (RichOutputFormat.class.isAssignableFrom(typedSink.getUserCodeWrapper().getUserCodeClass())) { ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : @@ -223,7 +215,7 @@ private List executeDataSource(GenericDataSourceBase source, in RuntimeUDFContext ctx; - MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, source.getName()); + MetricGroup metrics = NonRegisteringMetricsGroup.get(); if (RichInputFormat.class.isAssignableFrom(typedSource.getUserCodeWrapper().getUserCodeClass())) { ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); @@ -249,7 +241,7 @@ private List executeUnaryOperator(SingleInputOperator op TaskInfo taskInfo = new TaskInfo(typedOp.getName(), 0, 1, 0); RuntimeUDFContext ctx; - MetricGroup metrics = this.jobMetricGroup.addTask(new AbstractID(), new AbstractID(), 0, typedOp.getName()); + MetricGroup metrics = NonRegisteringMetricsGroup.get(); if (RichFunction.class.isAssignableFrom(typedOp.getUserCodeWrapper().getUserCodeClass())) { ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); @@ -291,7 +283,8 @@ private List executeBinaryOperator(DualInputOperator implements Metric { + /** * Calculates and returns the measured value. * diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java index 403612958e7d5..a3832ff09f77e 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java @@ -15,31 +15,49 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics; import org.apache.flink.annotation.PublicEvolving; /** - * A MetricGroup is a named container for {@link org.apache.flink.metrics.Metric}s and - * {@link org.apache.flink.metrics.MetricGroup}s. - *

- * Instances of this class can be used to register new metrics with Flink and to create a nested hierarchy based on the - * group names. - *

- * A MetricGroup is uniquely identified by it's place in the hierarchy and name. + * A MetricGroup is a named container for {@link Metric Metrics} and {@link MetricGroup MetricGroups}. + * + *

Instances of this class can be used to register new metrics with Flink and to create a nested + * hierarchy based on the group names. + * + *

A MetricGroup is uniquely identified by it's place in the hierarchy and name. + * + *

Metrics groups can be {@link #close() closed}. Upon closing, they de-register all metrics + * from any metrics reporter and any internal maps. Note that even closed metrics groups + * return Counters, Gauges, etc to the code, to prevent exceptions in the monitored code. + * These metrics simply do not get reported any more, when created on a closed group. */ @PublicEvolving public interface MetricGroup { + // ------------------------------------------------------------------------ + // Closing + // ------------------------------------------------------------------------ + /** - * Recursively unregisters all {@link org.apache.flink.metrics.Metric}s contained in this - * {@link org.apache.flink.metrics.MetricGroup} + * Marks the group as closed. + * Recursively unregisters all {@link Metric Metrics} contained in this group. + * + *

Any metrics created after the call to this function will not be registered in + * the {@link MetricRegistry} and not be reported to any reporter (like JMX). */ void close(); - // ----------------------------------------------------------------------------------------------------------------- - // Metrics - // ----------------------------------------------------------------------------------------------------------------- + /** + * Checks whether this MetricGroup has been closed. + * @return True if the group has been closed, false is the group is still open. + */ + boolean isClosed(); + + // ------------------------------------------------------------------------ + // Metrics + // ------------------------------------------------------------------------ /** * Creates and registers a new {@link org.apache.flink.metrics.Counter} with Flink. @@ -77,12 +95,12 @@ public interface MetricGroup { */ Gauge gauge(String name, Gauge gauge); - // ----------------------------------------------------------------------------------------------------------------- + // ------------------------------------------------------------------------ // Groups - // ----------------------------------------------------------------------------------------------------------------- + // ------------------------------------------------------------------------ /** - * Creates a new {@link org.apache.flink.metrics.MetricGroup} and adds it to this groups sub-groups. + * Creates a new MetricGroup and adds it to this groups sub-groups. * * @param name name of the group * @return the created group @@ -90,7 +108,7 @@ public interface MetricGroup { MetricGroup addGroup(int name); /** - * Creates a new {@link org.apache.flink.metrics.MetricGroup} and adds it to this groups sub-groups. + * Creates a new MetricGroup and adds it to this groups sub-groups. * * @param name name of the group * @return the created group diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java index 0e8b0d5bfecd6..7e062176ce1e6 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics; import org.apache.flink.annotation.Internal; @@ -24,6 +25,7 @@ import org.apache.flink.metrics.reporter.JMXReporter; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,15 +38,15 @@ import static org.apache.flink.metrics.groups.TaskMetricGroup.DEFAULT_SCOPE_TASK; /** - * A MetricRegistry keeps track of all registered {@link org.apache.flink.metrics.Metric}s. It serves as the - * connection between {@link org.apache.flink.metrics.MetricGroup}s and {@link org.apache.flink.metrics.reporter.MetricReporter}s. + * A MetricRegistry keeps track of all registered {@link Metric Metrics}. It serves as the + * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. */ @Internal public class MetricRegistry { - private static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); - private final MetricReporter reporter; - private java.util.Timer timer; + // ------------------------------------------------------------------------ + // configuration keys + // ------------------------------------------------------------------------ public static final String KEY_METRICS_REPORTER_CLASS = "metrics.reporter.class"; public static final String KEY_METRICS_REPORTER_ARGUMENTS = "metrics.reporter.arguments"; @@ -55,78 +57,88 @@ public class MetricRegistry { public static final String KEY_METRICS_SCOPE_NAMING_TASK = "metrics.scope.task"; public static final String KEY_METRICS_SCOPE_NAMING_OPERATOR = "metrics.scope.operator"; + // ------------------------------------------------------------------------ + // configuration keys + // ------------------------------------------------------------------------ + + private static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); + + private final MetricReporter reporter; + private final java.util.Timer timer; + private final Scope.ScopeFormat scopeConfig; /** - * Creates a new {@link MetricRegistry} and starts the configured reporter. + * Creates a new MetricRegistry and starts the configured reporter. */ public MetricRegistry(Configuration config) { + // first parse the scope formats, these are needed for all reporters + + Scope.ScopeFormat scopeFormat; try { - String className = config.getString(KEY_METRICS_REPORTER_CLASS, null); - if (className == null) { - LOG.info("No reporter class name defined in flink-conf.yaml, defaulting to " + JMXReporter.class.getName() + "."); - className = JMXReporter.class.getName(); - } + scopeFormat = createScopeConfig(config); + } + catch (Exception e) { + scopeFormat = createScopeConfig(new Configuration()); + LOG.warn("Failed to parse scope format, using default scope formats", e); + } + this.scopeConfig = scopeFormat; - this.scopeConfig = createScopeConfig(config); - - Configuration reporterConfig = createReporterConfig(config); - Class reporterClass = Class.forName(className); - reporter = (MetricReporter) reporterClass.newInstance(); - reporter.open(reporterConfig); - - if (reporter instanceof Scheduled) { - String[] interval = config.getString(KEY_METRICS_REPORTER_INTERVAL, "10 SECONDS").split(" "); - long millis = TimeUnit.valueOf(interval[1]).toMillis(Long.parseLong(interval[0])); - timer = new java.util.Timer(true); - timer.schedule(new TimerTask() { - @Override - public void run() { - ((Scheduled) reporter).report(); + // second, instantiate any custom configured reporters + + final String className = config.getString(KEY_METRICS_REPORTER_CLASS, null); + if (className == null) { + this.reporter = new JMXReporter(); + this.timer = null; + } + else { + MetricReporter reporter; + java.util.Timer timer; + + try { + String configuredPeriod = config.getString(KEY_METRICS_REPORTER_INTERVAL, null); + TimeUnit timeunit = TimeUnit.SECONDS; + long period = 10; + + if (configuredPeriod != null) { + try { + String[] interval = configuredPeriod.split(" "); + period = Long.parseLong(interval[0]); + timeunit = TimeUnit.valueOf(interval[1]); + } + catch (Exception e) { + LOG.error("Cannot parse report interval from config: " + configuredPeriod + + " - please use values like '10 SECONDS' or '500 MILLISECONDS'. " + + "Using default reporting interval."); } - }, millis, millis); + } + + Configuration reporterConfig = createReporterConfig(config, timeunit, period); + + Class reporterClass = Class.forName(className); + reporter = (MetricReporter) reporterClass.newInstance(); + reporter.open(reporterConfig); + + if (reporter instanceof Scheduled) { + LOG.info("Periodically reporting metrics in intervals of {} {}", period, timeunit.name()); + long millis = timeunit.toMillis(period); + + timer = new java.util.Timer("Periodic Metrics Reporter", true); + timer.schedule(new ReporterTask((Scheduled) reporter), millis, millis); + } + else { + timer = null; + } } - } catch (InstantiationException | ClassNotFoundException e) { - throw new RuntimeException("Error while instantiating reporter.", e); - } catch (IllegalAccessException e) { - throw new RuntimeException("Implementation error.", e); - } - } - - private static Configuration createReporterConfig(Configuration config) { - String[] interval = config.getString(KEY_METRICS_REPORTER_INTERVAL, "10 SECONDS").split(" "); - - String[] arguments = config.getString(KEY_METRICS_REPORTER_ARGUMENTS, "").split(" "); - - Configuration reporterConfig = new Configuration(); - reporterConfig.setString("period", interval[0]); - reporterConfig.setString("timeunit", interval[1]); - - if (arguments.length > 1) { - for (int x = 0; x < arguments.length; x += 2) { - reporterConfig.setString(arguments[x].replace("--", ""), arguments[x + 1]); + catch (Throwable t) { + reporter = new JMXReporter(); + timer = null; + LOG.error("Could not instantiate custom metrics reporter. Defaulting to JMX metrics export.", t); } - } - return reporterConfig; - } - - private static Scope.ScopeFormat createScopeConfig(Configuration config) { - String tmFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TM, DEFAULT_SCOPE_TM); - String jobFormat = config.getString(KEY_METRICS_SCOPE_NAMING_JOB, DEFAULT_SCOPE_JOB); - String taskFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TASK, DEFAULT_SCOPE_TASK); - String operatorFormat = config.getString(KEY_METRICS_SCOPE_NAMING_OPERATOR, DEFAULT_SCOPE_OPERATOR); - - Scope.ScopeFormat format = new Scope.ScopeFormat(); - format.setTaskManagerFormat(tmFormat); - format.setJobFormat(jobFormat); - format.setTaskFormat(taskFormat); - format.setOperatorFormat(operatorFormat); - return format; - } - - public Scope.ScopeFormat getScopeConfig() { - return this.scopeConfig; + this.reporter = reporter; + this.timer = timer; + } } /** @@ -137,10 +149,22 @@ public void shutdown() { timer.cancel(); } if (reporter != null) { - reporter.close(); + try { + reporter.close(); + } catch (Throwable t) { + LOG.warn("Metrics reporter did not shut down cleanly", t); + } } } + public Scope.ScopeFormat getScopeConfig() { + return this.scopeConfig; + } + + // ------------------------------------------------------------------------ + // Metrics (de)registration + // ------------------------------------------------------------------------ + /** * Registers a new {@link org.apache.flink.metrics.Metric} with this registry. * @@ -150,8 +174,6 @@ public void shutdown() { */ public void register(Metric metric, String name, AbstractMetricGroup parent) { String metricName = reporter.generateName(name, parent.generateScope()); - - this.reporter.notifyOfAddedMetric(metric, metricName); } @@ -167,4 +189,63 @@ public void unregister(Metric metric, String name, AbstractMetricGroup parent) { this.reporter.notifyOfRemovedMetric(metric, metricName); } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private static Configuration createReporterConfig(Configuration config, TimeUnit timeunit, long period) { + Configuration reporterConfig = new Configuration(); + reporterConfig.setLong("period", period); + reporterConfig.setString("timeunit", timeunit.name()); + + String[] arguments = config.getString(KEY_METRICS_REPORTER_ARGUMENTS, "").split(" "); + if (arguments.length > 1) { + for (int x = 0; x < arguments.length; x += 2) { + reporterConfig.setString(arguments[x].replace("--", ""), arguments[x + 1]); + } + } + return reporterConfig; + } + + private static Scope.ScopeFormat createScopeConfig(Configuration config) { + String tmFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TM, DEFAULT_SCOPE_TM); + String jobFormat = config.getString(KEY_METRICS_SCOPE_NAMING_JOB, DEFAULT_SCOPE_JOB); + String taskFormat = config.getString(KEY_METRICS_SCOPE_NAMING_TASK, DEFAULT_SCOPE_TASK); + String operatorFormat = config.getString(KEY_METRICS_SCOPE_NAMING_OPERATOR, DEFAULT_SCOPE_OPERATOR); + + + Scope.ScopeFormat format = new Scope.ScopeFormat(); + format.setTaskManagerFormat(tmFormat); + format.setJobFormat(jobFormat); + format.setTaskFormat(taskFormat); + format.setOperatorFormat(operatorFormat); + return format; + } + + // ------------------------------------------------------------------------ + + /** + * This task is explicitly a static class, so that it does not hold any references to the enclosing + * MetricsRegistry instance. + * + * This is a subtle difference, but very important: With this static class, the enclosing class instance + * may become garbage-collectible, whereas with an anonymous inner class, the timer thread + * (which is a GC root) will hold a reference via the timer task and its enclosing instance pointer. + * Making the MetricsRegistry garbage collectible makes the java.util.Timer garbage collectible, + * which acts as a fail-safe to stop the timer thread and prevents resource leaks. + */ + private static final class ReporterTask extends TimerTask { + + private final Scheduled reporter; + + private ReporterTask(Scheduled reporter) { + this.reporter = reporter; + } + + @Override + public void run() { + reporter.report(); + } + } } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java index 373ac0945cb8a..cad241d386aec 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/AbstractMetricGroup.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.groups; import org.apache.flink.annotation.Internal; @@ -23,44 +24,94 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.MetricRegistry; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Abstract {@link org.apache.flink.metrics.MetricGroup} that contains key functionality for adding metrics and groups. + * Abstract {@link MetricGroup} that contains key functionality for adding metrics and groups. + * + *

IMPORTANT IMPLEMENTATION NOTE + * + *

This class uses locks for adding and removing metrics objects. This is done to + * prevent resource leaks in the presence of concurrently closing a group and adding + * metrics and subgroups. + * Since closing groups recursively closes the subgroups, the lock acquisition order must + * be strictly from parent group to subgroup. If at any point, a subgroup holds its group + * lock and calls a parent method that also acquires the lock, it will create a deadlock + * condition. */ @Internal public abstract class AbstractMetricGroup implements MetricGroup { + + /** shared logger */ private static final Logger LOG = LoggerFactory.getLogger(MetricGroup.class); + + private static final String METRIC_NAME_REGEX = "[a-zA-Z0-9]*"; + + /** The pattern that metric and group names have to match */ + private static final Pattern METRIC_NAME_PATTERN = Pattern.compile(METRIC_NAME_REGEX); + + // ------------------------------------------------------------------------ + + /** The registry that this metrics group belongs to */ protected final MetricRegistry registry; - // all metrics that are directly contained in this group + /** All metrics that are directly contained in this group */ protected final Map metrics = new HashMap<>(); - // all generic groups that are directly contained in this group + + /** All metric subgroups of this group */ protected final Map groups = new HashMap<>(); + /** Flag indicating whether this group has been closed */ + private volatile boolean closed; + + // ------------------------------------------------------------------------ + public AbstractMetricGroup(MetricRegistry registry) { - this.registry = registry; + this.registry = checkNotNull(registry); } + // ------------------------------------------------------------------------ + // Closing + // ------------------------------------------------------------------------ + @Override public void close() { - for (MetricGroup group : groups.values()) { - group.close(); - } - this.groups.clear(); - for (Map.Entry metric : metrics.entrySet()) { - registry.unregister(metric.getValue(), metric.getKey(), this); + synchronized (this) { + if (!closed) { + closed = true; + + // close all subgroups + for (MetricGroup group : groups.values()) { + group.close(); + } + groups.clear(); + + // un-register all directly contained metrics + for (Map.Entry metric : metrics.entrySet()) { + registry.unregister(metric.getValue(), metric.getKey(), this); + } + metrics.clear(); + } } - this.metrics.clear(); + } + + @Override + public final boolean isClosed() { + return closed; } // ----------------------------------------------------------------------------------------------------------------- - // Scope + // Scope // ----------------------------------------------------------------------------------------------------------------- /** @@ -79,12 +130,12 @@ public void close() { public abstract List generateScope(Scope.ScopeFormat format); // ----------------------------------------------------------------------------------------------------------------- - // Metrics + // Metrics // ----------------------------------------------------------------------------------------------------------------- @Override public Counter counter(int name) { - return counter("" + name); + return counter(String.valueOf(name)); } @Override @@ -96,7 +147,7 @@ public Counter counter(String name) { @Override public Gauge gauge(int name, Gauge gauge) { - return gauge("" + name, gauge); + return gauge(String.valueOf(name), gauge); } @Override @@ -105,41 +156,91 @@ public Gauge gauge(String name, Gauge gauge) { return gauge; } - protected MetricGroup addMetric(String name, Metric metric) { - if (!name.matches("[a-zA-Z0-9]*")) { - throw new IllegalArgumentException("Metric names may not contain special characters."); - } - if (metrics.containsKey(name)) { - LOG.warn("Detected metric name collision. This group already contains a group for the given group name. " + - this.generateScope().toString() + "." + name); + /** + * Adds the given metric to the group and registers it at the registry, if the group + * is not yet closed, and if no metric with the same name has been registered before. + * + * @param name the name to register the metric under + * @param metric the metric to register + */ + protected void addMetric(String name, Metric metric) { + Matcher nameMatcher = METRIC_NAME_PATTERN.matcher(name); + if (!nameMatcher.matches()) { + throw new IllegalArgumentException("Metric names may not contain special characters or spaces. " + + "Allowed is: " + METRIC_NAME_REGEX); } - if (groups.containsKey(name)) { - LOG.warn("Detected metric name collision. This group already contains a group for the given metric name." + - this.generateScope().toString() + ")." + name); + + // add the metric only if the group is still open + synchronized (this) { + if (!closed) { + // immediately put without a 'contains' check to optimize the common case (no collition) + // collisions are resolved later + Metric prior = metrics.put(name, metric); + + // check for collisions with other metric names + if (prior == null) { + // no other metric with this name yet + + if (groups.containsKey(name)) { + // we warn here, rather than failing, because metrics are tools that should not fail the + // program when used incorrectly + LOG.warn("Name collision: Adding a metric with the same name as a metric subgroup: '" + + name + "'. Metric might not get properly reported. (" + generateScope() + ')'); + } + + registry.register(metric, name, this); + } + else { + // we had a collision. put back the original value + metrics.put(name, prior); + + // we warn here, rather than failing, because metrics are tools that should not fail the + // program when used incorrectly + LOG.warn("Name collision: Group already contains a Metric with the name '" + + name + "'. Metric will not be reported. (" + generateScope() + ')'); + } + } } - metrics.put(name, metric); - registry.register(metric, name, this); - return this; } - // ----------------------------------------------------------------------------------------------------------------- - // Groups - // ----------------------------------------------------------------------------------------------------------------- + // ------------------------------------------------------------------------ + // Groups + // ------------------------------------------------------------------------ @Override public MetricGroup addGroup(int name) { - return addGroup("" + name); + return addGroup(String.valueOf(name)); } @Override public MetricGroup addGroup(String name) { - if (metrics.containsKey(name)) { - LOG.warn("Detected metric name collision. This group already contains a metric for the given group name." - + this.generateScope().toString() + "." + name); - } - if (!groups.containsKey(name)) { - groups.put(name, new GenericMetricGroup(registry, this, name)); + synchronized (this) { + if (!closed) { + // adding a group with the same name as a metric creates problems in many reporters/dashboards + // we warn here, rather than failing, because metrics are tools that should not fail the + // program when used incorrectly + if (metrics.containsKey(name)) { + LOG.warn("Name collision: Adding a metric subgroup with the same name as an existing metric: '" + + name + "'. Metric might not get properly reported. (" + generateScope() + ')'); + } + + MetricGroup newGroup = new GenericMetricGroup(registry, this, name); + MetricGroup prior = groups.put(name, newGroup); + if (prior == null) { + // no prior group with that name + return newGroup; + } else { + // had a prior group with that name, add the prior group back + groups.put(name, prior); + return prior; + } + } + else { + // return a non-registered group that is immediately closed already + GenericMetricGroup closedGroup = new GenericMetricGroup(registry, this, name); + closedGroup.close(); + return closedGroup; + } } - return groups.get(name); } } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java index 81851e2e767f4..c68cdc1e82c90 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/ComponentMetricGroup.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.groups; import org.apache.flink.annotation.Internal; @@ -28,17 +29,26 @@ import static org.apache.flink.metrics.groups.Scope.SCOPE_WILDCARD; /** - * Abstract {@link org.apache.flink.metrics.MetricGroup} that contains key functionality for modifying the scope. + * Abstract {@link org.apache.flink.metrics.MetricGroup} for system components (e.g., + * TaskManager, Job, Task, Operator). + * + *

The components metric groups contain functionality to define alias names for + * the component identifier. For example, while Tasks are registered under a Task Attempt ID, + * the metrics name scope may use the task name instead. Using these aliases makes + * Metric scope names stable across jobs, recovery attempts, etc. */ @Internal public abstract class ComponentMetricGroup extends AbstractMetricGroup { - private final ComponentMetricGroup parent; - private final String format; + private final ComponentMetricGroup parent; + private final String format; + // Map: scope variable -> specific value protected final Map formats; + // ------------------------------------------------------------------------ + /** * Creates a new ComponentMetricGroup. * @@ -53,9 +63,45 @@ public ComponentMetricGroup(MetricRegistry registry, ComponentMetricGroup parent this.format = scopeFormat; } + /** + * Closes the component group by removing and closing all metrics and subgroups + * (inherited from {@link AbstractMetricGroup}), plus closing and removing all dedicated + * component subgroups. + */ + @Override + public void close() { + synchronized (this) { + if (!isClosed()) { + // remove all metrics and generic subgroups + super.close(); + + // remove and close all subcomponent metrics + for (ComponentMetricGroup group : subComponents()) { + group.close(); + } + } + } + } + + // ------------------------------------------------------------------------ + // sub components + // ------------------------------------------------------------------------ + + protected ComponentMetricGroup parent() { + return parent; + } + + protected abstract Iterable subComponents(); + + // ------------------------------------------------------------------------ + // scope format + // ------------------------------------------------------------------------ + + protected abstract String getScopeFormat(Scope.ScopeFormat format); + @Override public List generateScope() { - return this.generateScope(this.format); + return generateScope(format); } @Override @@ -63,8 +109,6 @@ public List generateScope(Scope.ScopeFormat format) { return generateScope(getScopeFormat(format)); } - protected abstract String getScopeFormat(Scope.ScopeFormat format); - private List generateScope(String format) { String[] components = Scope.split(format); diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java index 5886312aca647..eedb0fa036124 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/GenericMetricGroup.java @@ -27,12 +27,13 @@ */ @Internal public class GenericMetricGroup extends AbstractMetricGroup { + private final AbstractMetricGroup parent; private final String name; protected GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, int name) { - this(registry, parent, "" + name); + this(registry, parent, String.valueOf(name)); } protected GenericMetricGroup(MetricRegistry registry, AbstractMetricGroup parent, String name) { diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java index 68d91c448712b..dea565051e19a 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/IOMetricGroup.java @@ -26,12 +26,13 @@ * Special {@link org.apache.flink.metrics.MetricGroup} that contains shareable pre-defined IO-related metrics. */ public class IOMetricGroup extends AbstractMetricGroup { + private final TaskMetricGroup parent; - private transient final Counter numBytesIn; - private transient final Counter numBytesOut; - private transient final Counter numRecordsIn; - private transient final Counter numRecordsOut; + private final Counter numBytesIn; + private final Counter numBytesOut; + private final Counter numRecordsIn; + private final Counter numRecordsOut; public IOMetricGroup(MetricRegistry registry, TaskMetricGroup parent) { super(registry); diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java index 35a01f8d30e0c..f4f634a0b6f1d 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/JobMetricGroup.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.groups; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.JobID; -import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.MetricRegistry; import org.apache.flink.util.AbstractID; @@ -27,41 +27,85 @@ import java.util.Map; import static org.apache.flink.metrics.groups.TaskManagerMetricGroup.DEFAULT_SCOPE_TM; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Special {@link org.apache.flink.metrics.MetricGroup} representing a Job. - * Contains extra logic for adding tasks. */ @Internal public class JobMetricGroup extends ComponentMetricGroup { + public static final String SCOPE_JOB_DESCRIPTOR = "job"; public static final String SCOPE_JOB_ID = Scope.format("job_id"); public static final String SCOPE_JOB_NAME = Scope.format("job_name"); public static final String DEFAULT_SCOPE_JOB_COMPONENT = Scope.concat(SCOPE_JOB_NAME); public static final String DEFAULT_SCOPE_JOB = Scope.concat(DEFAULT_SCOPE_TM, DEFAULT_SCOPE_JOB_COMPONENT); - private Map tasks = new HashMap<>(); + // ------------------------------------------------------------------------ + + /** Map from execution attempt ID (task identifier) to task metrics */ + private final Map tasks = new HashMap<>(); + + private final JobID jobId; - public JobMetricGroup(MetricRegistry registry, TaskManagerMetricGroup taskManager, JobID id, String name) { + // ------------------------------------------------------------------------ + + public JobMetricGroup(MetricRegistry registry, TaskManagerMetricGroup taskManager, JobID jobId, String jobName) { super(registry, taskManager, registry.getScopeConfig().getJobFormat()); - this.formats.put(SCOPE_JOB_ID, id.toString()); - this.formats.put(SCOPE_JOB_NAME, name); + + this.jobId = checkNotNull(jobId); + this.formats.put(SCOPE_JOB_ID, jobId.toString()); + this.formats.put(SCOPE_JOB_NAME, checkNotNull(jobName)); + } + + // ------------------------------------------------------------------------ + // adding / removing tasks + // ------------------------------------------------------------------------ + + public TaskMetricGroup addTask(AbstractID vertexId, AbstractID executionId, int subtaskIndex, String name) { + checkNotNull(vertexId); + checkNotNull(executionId); + checkNotNull(name); + + synchronized (this) { + if (!isClosed()) { + TaskMetricGroup task = new TaskMetricGroup(registry, this, vertexId, executionId, subtaskIndex, name); + tasks.put(executionId, task); + return task; + } else { + return null; + } + } } - public TaskMetricGroup addTask(AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { - TaskMetricGroup task = new TaskMetricGroup(this.registry, this, id, attemptID, subtaskIndex, name); - tasks.put(id, task); - return task; + public void removeTaskMetricGroup(AbstractID executionId) { + checkNotNull(executionId); + + boolean removeFromParent = false; + synchronized (this) { + if (!isClosed() && tasks.remove(executionId) != null && tasks.isEmpty()) { + // this call removed the last task. close this group. + removeFromParent = true; + close(); + } + } + + // IMPORTANT: removing from the parent must happen while holding the this group's lock, + // because it would violate the "first parent then subgroup" lock acquisition order + if (removeFromParent) { + ((TaskManagerMetricGroup) parent()).removeJobMetricsGroup(jobId, this); + } } + // ------------------------------------------------------------------------ + // component group behavior + // ------------------------------------------------------------------------ + @Override - public void close() { - super.close(); - for (MetricGroup group : tasks.values()) { - group.close(); - } - tasks.clear(); + protected Iterable subComponents() { + return tasks.values(); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/NonRegisteringMetricsGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/NonRegisteringMetricsGroup.java new file mode 100644 index 0000000000000..1bfcfe32c16a8 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/NonRegisteringMetricsGroup.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.metrics.groups; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; + +/** + * Metrics group that does not register any metrics. + */ +@Internal +public class NonRegisteringMetricsGroup implements MetricGroup { + + // ------------------------------------------------------------------------ + // singleton + // ------------------------------------------------------------------------ + + private static final NonRegisteringMetricsGroup INSTANCE = new NonRegisteringMetricsGroup(); + + public static NonRegisteringMetricsGroup get() { + return INSTANCE; + } + + /** Private constructor to prevent instantiation */ + private NonRegisteringMetricsGroup() {} + + // ------------------------------------------------------------------------ + // metrics group + // ------------------------------------------------------------------------ + + @Override + public void close() {} + + @Override + public boolean isClosed() { + return false; + } + + @Override + public Counter counter(int name) { + return new Counter(); + } + + @Override + public Counter counter(String name) { + return new Counter(); + } + + @Override + public Gauge gauge(int name, Gauge gauge) { + return gauge; + } + + @Override + public Gauge gauge(String name, Gauge gauge) { + return gauge; + } + + + @Override + public MetricGroup addGroup(int name) { + return addGroup(String.valueOf(name)); + } + + @Override + public MetricGroup addGroup(String name) { + return new NonRegisteringMetricsGroup(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java index 6475eecc31a8d..390b55b68ec3b 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/OperatorMetricGroup.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.metrics.MetricRegistry; +import java.util.Collections; + import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; /** @@ -27,6 +29,7 @@ */ @Internal public class OperatorMetricGroup extends ComponentMetricGroup { + public static final String SCOPE_OPERATOR_DESCRIPTOR = "operator"; public static final String SCOPE_OPERATOR_NAME = Scope.format("operator_name"); public static final String SCOPE_OPERATOR_SUBTASK_INDEX = Scope.format("subtask_index"); @@ -35,12 +38,20 @@ public class OperatorMetricGroup extends ComponentMetricGroup { protected OperatorMetricGroup(MetricRegistry registry, TaskMetricGroup task, String name, int subTaskIndex) { super(registry, task, registry.getScopeConfig().getOperatorFormat()); + this.formats.put(SCOPE_OPERATOR_NAME, name); - this.formats.put(SCOPE_OPERATOR_SUBTASK_INDEX, "" + subTaskIndex); + this.formats.put(SCOPE_OPERATOR_SUBTASK_INDEX, String.valueOf(subTaskIndex)); } + // ------------------------------------------------------------------------ + @Override protected String getScopeFormat(Scope.ScopeFormat format) { return format.getOperatorFormat(); } + + @Override + protected Iterable subComponents() { + return Collections.emptyList(); + } } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java index 47bae37ec4641..83013e2e9c746 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/Scope.java @@ -73,13 +73,12 @@ public static String[] split(String scope) { * Simple container for component scope format strings. */ public static class ScopeFormat { + private String operatorFormat = OperatorMetricGroup.DEFAULT_SCOPE_OPERATOR; private String taskFormat = TaskMetricGroup.DEFAULT_SCOPE_TASK; private String jobFormat = JobMetricGroup.DEFAULT_SCOPE_JOB; private String taskManagerFormat = TaskManagerMetricGroup.DEFAULT_SCOPE_TM; - - public ScopeFormat() { - } + public ScopeFormat setOperatorFormat(String format) { this.operatorFormat = format; diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java index e199ca7ea76c1..bfb9362387977 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskManagerMetricGroup.java @@ -15,23 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.groups; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.JobID; -import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; import java.util.HashMap; import java.util.Map; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Special {@link org.apache.flink.metrics.MetricGroup} representing a TaskManager. - * Contains extra logic for adding jobs with tasks, and removing jobs when they do + * not contain tasks any more */ @Internal public class TaskManagerMetricGroup extends ComponentMetricGroup { + public static final String SCOPE_HOST_DESCRIPTOR = "host"; public static final String SCOPE_TM_DESCRIPTOR = "taskmanager"; public static final String SCOPE_TM_HOST = Scope.format("host"); @@ -39,32 +44,87 @@ public class TaskManagerMetricGroup extends ComponentMetricGroup { public static final String DEFAULT_SCOPE_TM_COMPONENT = Scope.concat(SCOPE_TM_HOST, "taskmanager", SCOPE_TM_ID); public static final String DEFAULT_SCOPE_TM = DEFAULT_SCOPE_TM_COMPONENT; - private Map jobs = new HashMap<>(); + // ------------------------------------------------------------------------ + + private final Map jobs = new HashMap<>(); - public TaskManagerMetricGroup(MetricRegistry registry, String host, String id) { + public TaskManagerMetricGroup(MetricRegistry registry, String host, String taskManagerId) { super(registry, null, registry.getScopeConfig().getTaskManagerFormat()); - this.formats.put(SCOPE_TM_HOST, host); - this.formats.put(SCOPE_TM_ID, id); - } - public JobMetricGroup addJob(JobID id, String name) { - JobMetricGroup task = new JobMetricGroup(this.registry, this, id, name); - jobs.put(id, task); - return task; + this.formats.put(SCOPE_TM_HOST, checkNotNull(host)); + this.formats.put(SCOPE_TM_ID, checkNotNull(taskManagerId)); } - @Override - public void close() { - super.close(); - for (MetricGroup group : jobs.values()) { - group.close(); + // ------------------------------------------------------------------------ + // job groups + // ------------------------------------------------------------------------ + + public TaskMetricGroup addTaskForJob( + JobID jobId, + String jobName, + AbstractID vertexID, + AbstractID executionId, + int subtaskIndex, + String taskName) { + + // we cannot strictly lock both our map modification and the job group modification + // because it might lead to a deadlock + while (true) { + // get or create a jobs metric group + JobMetricGroup currentJobGroup; + synchronized (this) { + currentJobGroup = jobs.get(jobId); + + if (currentJobGroup == null || currentJobGroup.isClosed()) { + currentJobGroup = new JobMetricGroup(registry, this, jobId, jobName); + jobs.put(jobId, currentJobGroup); + } + } + + // try to add another task. this may fail if we found a pre-existing job metrics + // group and it is closed concurrently + TaskMetricGroup taskGroup = currentJobGroup.addTask(vertexID, executionId, subtaskIndex, taskName); + if (taskGroup != null) { + // successfully added the next task + return taskGroup; + } + + // else fall through the loop + } + } + + public void removeJobMetricsGroup(JobID jobId, JobMetricGroup group) { + if (jobId == null || group == null || !group.isClosed()) { + return; + } + + synchronized (this) { + // optimistically remove the currently contained group, and check later if it was correct + JobMetricGroup containedGroup = jobs.remove(jobId); + + // check if another group was actually contained, and restore that one + if (containedGroup != null && containedGroup != group) { + jobs.put(jobId, containedGroup); + } } - jobs.clear(); } + public int numRegisteredJobMetricGroups() { + return jobs.size(); + } + + // ------------------------------------------------------------------------ + // component group behavior + // ------------------------------------------------------------------------ + @Override protected String getScopeFormat(Scope.ScopeFormat format) { return format.getTaskManagerFormat(); } + + @Override + protected Iterable subComponents() { + return jobs.values(); + } } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java index 4f8e010a528f7..323853478d179 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.groups; import org.apache.flink.annotation.Internal; -import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.MetricRegistry; import org.apache.flink.util.AbstractID; @@ -26,14 +26,16 @@ import java.util.Map; import static org.apache.flink.metrics.groups.JobMetricGroup.DEFAULT_SCOPE_JOB; +import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Special {@link org.apache.flink.metrics.MetricGroup} representing a Task. - * Contains extra logic for adding operators. */ @Internal public class TaskMetricGroup extends ComponentMetricGroup { + public static final String SCOPE_TASK_DESCRIPTOR = "task"; public static final String SCOPE_TASK_ID = Scope.format("task_id"); public static final String SCOPE_TASK_NAME = Scope.format("task_name"); @@ -41,34 +43,56 @@ public class TaskMetricGroup extends ComponentMetricGroup { public static final String SCOPE_TASK_SUBTASK_INDEX = Scope.format("subtask_index"); public static final String DEFAULT_SCOPE_TASK_COMPONENT = SCOPE_TASK_NAME; public static final String DEFAULT_SCOPE_TASK = Scope.concat(DEFAULT_SCOPE_JOB, DEFAULT_SCOPE_TASK_COMPONENT); + + + private final Map operators = new HashMap<>(); + + private final IOMetricGroup ioMetrics; + + private final AbstractID executionId; + private final int subtaskIndex; - private Map operators = new HashMap<>(); - private IOMetricGroup ioMetrics; + protected TaskMetricGroup( + MetricRegistry registry, + JobMetricGroup parent, + AbstractID taskId, + AbstractID executionId, + int subtaskIndex, + String name) { + + super(registry, parent, registry.getScopeConfig().getTaskFormat()); - protected TaskMetricGroup(MetricRegistry registry, JobMetricGroup job, AbstractID id, AbstractID attemptID, int subtaskIndex, String name) { - super(registry, job, registry.getScopeConfig().getTaskFormat()); - this.formats.put(SCOPE_TASK_ID, id.toString()); - this.formats.put(SCOPE_TASK_ATTEMPT, attemptID.toString()); - this.formats.put(SCOPE_TASK_NAME, name); - this.formats.put(SCOPE_TASK_SUBTASK_INDEX, "" + subtaskIndex); + this.executionId = executionId; this.subtaskIndex = subtaskIndex; this.ioMetrics = new IOMetricGroup(registry, this); + + this.formats.put(SCOPE_TASK_ID, taskId.toString()); + this.formats.put(SCOPE_TASK_ATTEMPT, executionId.toString()); + this.formats.put(SCOPE_TASK_NAME, checkNotNull(name)); + this.formats.put(SCOPE_TASK_SUBTASK_INDEX, String.valueOf(subtaskIndex)); } - @Override - public void close() { - super.close(); - for (MetricGroup group : operators.values()) { - group.close(); + public OperatorMetricGroup addOperator(String name) { + OperatorMetricGroup operator = new OperatorMetricGroup(this.registry, this, name, this.subtaskIndex); + + synchronized (this) { + OperatorMetricGroup previous = operators.put(name, operator); + if (previous == null) { + // no operator group so far + return operator; + } else { + // already had an operator group. restore that one. + operators.put(name, previous); + return previous; + } } - operators.clear(); } - public OperatorMetricGroup addOperator(String name) { - OperatorMetricGroup operator = new OperatorMetricGroup(this.registry, this, name, this.subtaskIndex); - operators.put(name, operator); - return operator; + @Override + public void close() { + super.close(); + parent().removeTaskMetricGroup(executionId); } /** @@ -80,8 +104,18 @@ public IOMetricGroup getIOMetricGroup() { return this.ioMetrics; } + @Override + protected JobMetricGroup parent() { + return (JobMetricGroup) super.parent(); + } + @Override protected String getScopeFormat(Scope.ScopeFormat format) { return format.getTaskFormat(); } + + @Override + protected Iterable subComponents() { + return operators.values(); + } } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java index 43f09b2293f16..1fe55df0b7377 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.reporter; import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; @@ -25,7 +26,8 @@ import java.util.Map; public abstract class AbstractReporter implements MetricReporter { - protected Map gauges = new ConcurrentHashMap<>(); + + protected Map> gauges = new ConcurrentHashMap<>(); protected Map counters = new ConcurrentHashMap<>(); @Override @@ -33,7 +35,7 @@ public void notifyOfAddedMetric(Metric metric, String name) { if (metric instanceof Counter) { counters.put(name, (Counter) metric); } else if (metric instanceof Gauge) { - gauges.put(name, (Gauge) metric); + gauges.put(name, (Gauge) metric); } } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java index 0b2efe4b5de4e..69fa11ad280e0 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/JMXReporter.java @@ -120,7 +120,7 @@ public String generateName(String name, List origin) { fullName.append(value); fullName.append(","); } - fullName.append("name=" + name); + fullName.append("name=").append(name); return fullName.toString(); } @@ -153,9 +153,10 @@ public interface JmxGaugeMBean extends MetricMBean { } private static class JmxGauge extends AbstractBean implements JmxGaugeMBean { - private final Gauge gauge; + + private final Gauge gauge; - public JmxGauge(Gauge gauge) { + public JmxGauge(Gauge gauge) { this.gauge = gauge; } diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java index 2bca606e215f1..f87f5d30eff40 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java @@ -31,6 +31,7 @@ */ @PublicEvolving public interface MetricReporter extends Reporter { + /** * Configures this reporter. Since reporters are instantiated generically and hence parameter-less, * this method is the place where the reporters set their basic fields based on configuration values. diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java index 89483b3dbcccf..cea11fb0a8a39 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/JobGroupTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; import org.junit.Test; import java.util.List; @@ -30,10 +31,12 @@ public class JobGroupTest { @Test public void testGenerateScopeDefault() { MetricRegistry registry = new MetricRegistry(new Configuration()); - JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job"); - List scope = operator.generateScope(); + TaskMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "host", "id") + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task"); + JobMetricGroup jmGroup = tmGroup.parent(); + + List scope = jmGroup.generateScope(); assertEquals(4, scope.size()); assertEquals("job", scope.get(3)); } @@ -41,13 +44,15 @@ public void testGenerateScopeDefault() { @Test public void testGenerateScopeWildcard() { MetricRegistry registry = new MetricRegistry(new Configuration()); - JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job"); + + TaskMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "host", "id") + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task"); + JobMetricGroup jmGroup = tmGroup.parent(); Scope.ScopeFormat format = new Scope.ScopeFormat(); format.setJobFormat(Scope.concat(Scope.SCOPE_WILDCARD, "superjob", JobMetricGroup.SCOPE_JOB_NAME)); - List scope = operator.generateScope(format); + List scope = jmGroup.generateScope(format); assertEquals(5, scope.size()); assertEquals("superjob", scope.get(3)); assertEquals("job", scope.get(4)); @@ -56,13 +61,15 @@ public void testGenerateScopeWildcard() { @Test public void testGenerateScopeCustom() { MetricRegistry registry = new MetricRegistry(new Configuration()); - JobMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job"); + + TaskMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "host", "id") + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task"); + JobMetricGroup jmGroup = tmGroup.parent(); Scope.ScopeFormat format = new Scope.ScopeFormat(); format.setJobFormat(Scope.concat(TaskManagerMetricGroup.SCOPE_TM_HOST, "superjob", JobMetricGroup.SCOPE_JOB_NAME)); - List scope = operator.generateScope(format); + List scope = jmGroup.generateScope(format); assertEquals(3, scope.size()); assertEquals("host", scope.get(0)); assertEquals("superjob", scope.get(1)); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupRegistrationTest.java similarity index 77% rename from flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java rename to flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupRegistrationTest.java index f7502e58dbb9b..cb962ea4eddf3 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/MetricGroupTest.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupRegistrationTest.java @@ -15,41 +15,49 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.metrics; +package org.apache.flink.metrics.groups; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.groups.TaskManagerMetricGroup; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; import org.apache.flink.metrics.util.TestReporter; + import org.junit.Assert; import org.junit.Test; -public class MetricGroupTest { +import static org.junit.Assert.*; + +public class MetricGroupRegistrationTest { /** * Verifies that group methods instantiate the correct metric with the given name. */ @Test public void testMetricInstantiation() { Configuration config = new Configuration(); - config.setString(MetricRegistry.KEY_METRICS_REPORTER_CLASS, TestReporter1.class.getName()); MetricGroup root = new TaskManagerMetricGroup(new MetricRegistry(config), "host", "id"); - root.counter("counter"); - Assert.assertTrue(TestReporter1.lastPassedMetric instanceof Counter); - Assert.assertEquals("counter", TestReporter1.lastPassedName); + Counter counter = root.counter("counter"); + assertEquals(counter, TestReporter1.lastPassedMetric); + assertEquals("counter", TestReporter1.lastPassedName); - root.gauge("gauge", new Gauge() { + Gauge gauge = root.gauge("gauge", new Gauge() { @Override public Object getValue() { return null; } }); - Assert.assertTrue(TestReporter1.lastPassedMetric instanceof Gauge); - Assert.assertEquals("gauge", TestReporter1.lastPassedName); + + Assert.assertEquals(gauge, TestReporter1.lastPassedMetric); + assertEquals("gauge", TestReporter1.lastPassedName); } - protected static class TestReporter1 extends TestReporter { + public static class TestReporter1 extends TestReporter { + public static Metric lastPassedMetric; public static String lastPassedName; @@ -60,8 +68,7 @@ public void notifyOfAddedMetric(Metric metric, String name) { } @Override - public void notifyOfRemovedMetric(Metric metric, String name) { - } + public void notifyOfRemovedMetric(Metric metric, String name) {} } /** diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupTest.java new file mode 100644 index 0000000000000..31ab2a7f29618 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/MetricGroupTest.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.metrics.groups; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; + +import org.apache.flink.metrics.util.DummyMetricGroup; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class MetricGroupTest { + + private final MetricRegistry registry = new MetricRegistry(new Configuration()); + + private final MetricRegistry exceptionOnRegister = new ExceptionOnRegisterRegistry(); + + @Test + public void sameGroupOnNameCollision() { + GenericMetricGroup group = new GenericMetricGroup(registry, new DummyMetricGroup(registry), "somegroup"); + + String groupName = "sometestname"; + MetricGroup subgroup1 = group.addGroup(groupName); + MetricGroup subgroup2 = group.addGroup(groupName); + + assertNotNull(subgroup1); + assertNotNull(subgroup2); + assertTrue(subgroup1 == subgroup2); + } + + @Test + public void closedGroupDoesNotRegisterMetrics() { + GenericMetricGroup group = new GenericMetricGroup( + exceptionOnRegister, new DummyMetricGroup(exceptionOnRegister), "testgroup"); + assertFalse(group.isClosed()); + + group.close(); + assertTrue(group.isClosed()); + + // these will fail is the registration is propagated + group.counter("testcounter"); + group.gauge("testgauge", new Gauge() { + @Override + public Object getValue() { return null; } + }); + } + + @Test + public void closedGroupCreatesClosedGroups() { + GenericMetricGroup group = new GenericMetricGroup(exceptionOnRegister, + new DummyMetricGroup(exceptionOnRegister), "testgroup"); + assertFalse(group.isClosed()); + + group.close(); + assertTrue(group.isClosed()); + + MetricGroup subgroup = group.addGroup("test subgroup"); + assertTrue(subgroup.isClosed()); + } + + @Test + public void tolerateMetricNameCollisions() { + final String name = "abctestname"; + GenericMetricGroup group = new GenericMetricGroup( + registry, new DummyMetricGroup(registry), "testgroup"); + + assertNotNull(group.counter(name)); + assertNotNull(group.counter(name)); + } + + @Test + public void tolerateMetricAndGroupNameCollisions() { + final String name = "abctestname"; + GenericMetricGroup group = new GenericMetricGroup(registry, new DummyMetricGroup(registry), "testgroup"); + + assertNotNull(group.addGroup(name)); + assertNotNull(group.counter(name)); + } + + @Test(expected = IllegalArgumentException.class) + public void exceptionOnIllegalName() { + GenericMetricGroup group = new GenericMetricGroup( + exceptionOnRegister, new DummyMetricGroup(exceptionOnRegister), "testgroup"); + group.counter("ÜberCöunter"); + } + + // ------------------------------------------------------------------------ + + private static class ExceptionOnRegisterRegistry extends MetricRegistry { + + public ExceptionOnRegisterRegistry() { + super(new Configuration()); + } + + @Override + public void register(Metric metric, String name, AbstractMetricGroup parent) { + fail("Metric should never be registered"); + } + + @Override + public void unregister(Metric metric, String name, AbstractMetricGroup parent) { + fail("Metric should never be un-registered"); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java index 4f33d2a54edf0..62028342fcd69 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/OperatorGroupTest.java @@ -31,9 +31,10 @@ public class OperatorGroupTest { @Test public void testGenerateScopeDefault() { MetricRegistry registry = new MetricRegistry(new Configuration()); + OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job") - .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task") + .addOperator("operator"); List scope = operator.generateScope(); assertEquals(6, scope.size()); @@ -49,8 +50,8 @@ public void testGenerateScopeDefault() { public void testGenerateScopeWildcard() { MetricRegistry registry = new MetricRegistry(new Configuration()); OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job") - .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task") + .addOperator("operator"); Scope.ScopeFormat format = new Scope.ScopeFormat(); format.setOperatorFormat(Scope.concat(Scope.SCOPE_WILDCARD, "op", OperatorMetricGroup.SCOPE_OPERATOR_NAME)); @@ -70,8 +71,8 @@ public void testGenerateScopeWildcard() { public void testGenerateScopeCustom() { MetricRegistry registry = new MetricRegistry(new Configuration()); OperatorMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job") - .addTask(new AbstractID(), new AbstractID(), 0, "task").addOperator("operator"); + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task") + .addOperator("operator"); Scope.ScopeFormat format = new Scope.ScopeFormat(); format.setOperatorFormat(Scope.concat("jobs", JobMetricGroup.SCOPE_JOB_NAME, "op", OperatorMetricGroup.SCOPE_OPERATOR_NAME)); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java index c49fdcdc95f1f..69faee7279d67 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskGroupTest.java @@ -28,12 +28,17 @@ import static org.junit.Assert.assertEquals; public class TaskGroupTest { + + // ------------------------------------------------------------------------ + // scope tests + // ------------------------------------------------------------------------ + @Test public void testGenerateScopeDefault() { MetricRegistry registry = new MetricRegistry(new Configuration()); + TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job") - .addTask(new AbstractID(), new AbstractID(), 0, "task"); + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task"); List scope = operator.generateScope(); assertEquals(5, scope.size()); @@ -44,8 +49,7 @@ public void testGenerateScopeDefault() { public void testGenerateScopeWilcard() { MetricRegistry registry = new MetricRegistry(new Configuration()); TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job") - .addTask(new AbstractID(), new AbstractID(), 0, "task"); + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task"); Scope.ScopeFormat format = new Scope.ScopeFormat(); format.setTaskFormat(Scope.concat(Scope.SCOPE_WILDCARD, "supertask", TaskMetricGroup.SCOPE_TASK_NAME)); @@ -64,8 +68,7 @@ public void testGenerateScopeWilcard() { public void testGenerateScopeCustom() { MetricRegistry registry = new MetricRegistry(new Configuration()); TaskMetricGroup operator = new TaskManagerMetricGroup(registry, "host", "id") - .addJob(new JobID(), "job") - .addTask(new AbstractID(), new AbstractID(), 0, "task"); + .addTaskForJob(new JobID(), "job", new AbstractID(), new AbstractID(), 0, "task"); Scope.ScopeFormat format = new Scope.ScopeFormat(); format.setTaskFormat(Scope.concat(TaskManagerMetricGroup.SCOPE_TM_HOST, JobMetricGroup.SCOPE_JOB_NAME, "supertask", TaskMetricGroup.SCOPE_TASK_NAME)); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java index 7b3286d05aacb..6b9a5fc5c8534 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/groups/TaskManagerGroupTest.java @@ -17,15 +17,112 @@ */ package org.apache.flink.metrics.groups; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.MetricRegistry; +import org.apache.flink.util.AbstractID; import org.junit.Test; import java.util.List; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.*; public class TaskManagerGroupTest { + + // ------------------------------------------------------------------------ + // adding and removing jobs + // ------------------------------------------------------------------------ + + @Test + public void addAndRemoveJobs() { + final TaskManagerMetricGroup group = new TaskManagerMetricGroup( + new MetricRegistry(new Configuration()), "localhost", new AbstractID().toString()); + + + final JobID jid1 = new JobID(); + final JobID jid2 = new JobID(); + + final String jobName1 = "testjob"; + final String jobName2 = "anotherJob"; + + final AbstractID vertex11 = new AbstractID(); + final AbstractID vertex12 = new AbstractID(); + final AbstractID vertex13 = new AbstractID(); + final AbstractID vertex21 = new AbstractID(); + + final AbstractID execution11 = new AbstractID(); + final AbstractID execution12 = new AbstractID(); + final AbstractID execution13 = new AbstractID(); + final AbstractID execution21 = new AbstractID(); + + TaskMetricGroup tmGroup11 = group.addTaskForJob(jid1, jobName1, vertex11, execution11, 17, "test"); + TaskMetricGroup tmGroup12 = group.addTaskForJob(jid1, jobName1, vertex12, execution12, 13, "test"); + TaskMetricGroup tmGroup21 = group.addTaskForJob(jid2, jobName2, vertex21, execution21, 7, "test"); + + assertEquals(2, group.numRegisteredJobMetricGroups()); + assertFalse(tmGroup11.parent().isClosed()); + assertFalse(tmGroup12.parent().isClosed()); + assertFalse(tmGroup21.parent().isClosed()); + + // close all for job 2 and one from job 1 + tmGroup11.close(); + tmGroup21.close(); + assertTrue(tmGroup11.isClosed()); + assertTrue(tmGroup21.isClosed()); + + // job 2 should be removed, job should still be there + assertFalse(tmGroup11.parent().isClosed()); + assertFalse(tmGroup12.parent().isClosed()); + assertTrue(tmGroup21.parent().isClosed()); + assertEquals(1, group.numRegisteredJobMetricGroups()); + + // add one more to job one + TaskMetricGroup tmGroup13 = group.addTaskForJob(jid1, jobName1, vertex13, execution13, 0, "test"); + tmGroup12.close(); + tmGroup13.close(); + + assertTrue(tmGroup11.parent().isClosed()); + assertTrue(tmGroup12.parent().isClosed()); + assertTrue(tmGroup13.parent().isClosed()); + + assertEquals(0, group.numRegisteredJobMetricGroups()); + } + + @Test + public void testCloseClosesAll() { + final TaskManagerMetricGroup group = new TaskManagerMetricGroup( + new MetricRegistry(new Configuration()), "localhost", new AbstractID().toString()); + + + final JobID jid1 = new JobID(); + final JobID jid2 = new JobID(); + + final String jobName1 = "testjob"; + final String jobName2 = "anotherJob"; + + final AbstractID vertex11 = new AbstractID(); + final AbstractID vertex12 = new AbstractID(); + final AbstractID vertex21 = new AbstractID(); + + final AbstractID execution11 = new AbstractID(); + final AbstractID execution12 = new AbstractID(); + final AbstractID execution21 = new AbstractID(); + + TaskMetricGroup tmGroup11 = group.addTaskForJob(jid1, jobName1, vertex11, execution11, 17, "test"); + TaskMetricGroup tmGroup12 = group.addTaskForJob(jid1, jobName1, vertex12, execution12, 13, "test"); + TaskMetricGroup tmGroup21 = group.addTaskForJob(jid2, jobName2, vertex21, execution21, 7, "test"); + + group.close(); + + assertTrue(tmGroup11.isClosed()); + assertTrue(tmGroup12.isClosed()); + assertTrue(tmGroup21.isClosed()); + } + + // ------------------------------------------------------------------------ + // scope tests + // ------------------------------------------------------------------------ + @Test public void testGenerateScopeDefault() { MetricRegistry registry = new MetricRegistry(new Configuration()); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java index d607072e8fea2..ab78288fd07d2 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyJobMetricGroup.java @@ -24,6 +24,7 @@ import org.apache.flink.util.AbstractID; public class DummyJobMetricGroup extends JobMetricGroup { + public DummyJobMetricGroup() { super(new DummyMetricRegistry(), new DummyTaskManagerMetricGroup(), new JobID(), "job"); } @@ -34,14 +35,10 @@ public DummyTaskMetricGroup addTask(AbstractID id, AbstractID attemptID, int sub } @Override - protected MetricGroup addMetric(String name, Metric metric) { - return this; - } + public void removeTaskMetricGroup(AbstractID executionId) {} @Override - public MetricGroup addGroup(int name) { - return addGroup("" + name); - } + protected void addMetric(String name, Metric metric) {} @Override public MetricGroup addGroup(String name) { diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java index 26df8749348f9..77ddd170959c1 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricGroup.java @@ -19,6 +19,7 @@ import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.MetricRegistry; import org.apache.flink.metrics.groups.AbstractMetricGroup; import org.apache.flink.metrics.groups.Scope; @@ -26,8 +27,13 @@ import java.util.List; public class DummyMetricGroup extends AbstractMetricGroup { + public DummyMetricGroup() { - super(new DummyMetricRegistry()); + this(new DummyMetricRegistry()); + } + + public DummyMetricGroup(MetricRegistry registry) { + super(registry); } @Override @@ -41,14 +47,8 @@ public List generateScope(Scope.ScopeFormat format) { } @Override - protected MetricGroup addMetric(String name, Metric metric) { - return this; - } - - @Override - public MetricGroup addGroup(int name) { - return addGroup("" + name); - } + protected void addMetric(String name, Metric metric) {} + @Override public MetricGroup addGroup(String name) { diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java index f0d6d3f4cd780..f8b73a91679b0 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyMetricRegistry.java @@ -15,18 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.util; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.MetricRegistry; public class DummyMetricRegistry extends MetricRegistry { - private static final Configuration config; - - static { - config = new Configuration(); - config.setString(KEY_METRICS_REPORTER_CLASS, DummyReporter.class.getCanonicalName()); - } public DummyMetricRegistry() { super(new Configuration()); diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java index eb45f6ae31cd4..e271d6ede740c 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyOperatorMetricGroup.java @@ -22,19 +22,13 @@ import org.apache.flink.metrics.groups.OperatorMetricGroup; public class DummyOperatorMetricGroup extends OperatorMetricGroup { + public DummyOperatorMetricGroup() { super(new DummyMetricRegistry(), new DummyTaskMetricGroup(), "operator", 0); } @Override - protected MetricGroup addMetric(String name, Metric metric) { - return this; - } - - @Override - public MetricGroup addGroup(int name) { - return addGroup("" + name); - } + protected void addMetric(String name, Metric metric) {} @Override public MetricGroup addGroup(String name) { diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java index 1c7d33bc4db64..8b7714ff2a509 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskManagerMetricGroup.java @@ -23,6 +23,7 @@ import org.apache.flink.metrics.groups.TaskManagerMetricGroup; public class DummyTaskManagerMetricGroup extends TaskManagerMetricGroup { + public DummyTaskManagerMetricGroup() { super(new DummyMetricRegistry(), "host", "id"); } @@ -32,14 +33,7 @@ public DummyJobMetricGroup addJob(JobID id, String name) { } @Override - protected MetricGroup addMetric(String name, Metric metric) { - return this; - } - - @Override - public MetricGroup addGroup(int name) { - return addGroup("" + name); - } + protected void addMetric(String name, Metric metric) {} @Override public MetricGroup addGroup(String name) { diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java index 53683f4f8430b..db2c5575f1411 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/DummyTaskMetricGroup.java @@ -23,6 +23,7 @@ import org.apache.flink.util.AbstractID; public class DummyTaskMetricGroup extends TaskMetricGroup { + public DummyTaskMetricGroup() { super(new DummyMetricRegistry(), new DummyJobMetricGroup(), new AbstractID(), new AbstractID(), 0, "task"); } @@ -32,14 +33,7 @@ public DummyOperatorMetricGroup addOperator(String name) { } @Override - protected MetricGroup addMetric(String name, Metric metric) { - return this; - } - - @Override - public MetricGroup addGroup(int name) { - return addGroup("" + name); - } + protected void addMetric(String name, Metric metric) {} @Override public MetricGroup addGroup(String name) { diff --git a/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java b/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java index 482d1e8aa22c7..5d8a8e0dd0d37 100644 --- a/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java +++ b/flink-core/src/test/java/org/apache/flink/metrics/util/TestReporter.java @@ -15,23 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.metrics.util; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.reporter.AbstractReporter; -import org.apache.flink.metrics.reporter.MetricReporter; import java.util.List; public class TestReporter extends AbstractReporter { + @Override - public void open(Configuration config) { - } + public void open(Configuration config) {} @Override - public void close() { - } + public void close() {} @Override public String generateName(String name, List scope) { diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metric-reporters/flink-metrics-dropwizard/pom.xml similarity index 97% rename from flink-metrics/flink-metrics-dropwizard/pom.xml rename to flink-metric-reporters/flink-metrics-dropwizard/pom.xml index 84d97228c08f6..a3868805640ee 100644 --- a/flink-metrics/flink-metrics-dropwizard/pom.xml +++ b/flink-metric-reporters/flink-metrics-dropwizard/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink - flink-metrics + flink-metric-reporters 1.1-SNAPSHOT .. diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java similarity index 97% rename from flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java rename to flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java index a7309be39aa9a..059704d32050d 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java +++ b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/ScheduledDropwizardReporter.java @@ -55,7 +55,7 @@ public synchronized void notifyOfAddedMetric(Metric metric, String name) { if (metric instanceof Counter) { registry.register(name, new CounterWrapper((Counter) metric)); } else if (metric instanceof Gauge) { - registry.register(name, new GaugeWrapper((Gauge) metric)); + registry.register(name, new GaugeWrapper((Gauge) metric)); } } diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java similarity index 100% rename from flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java rename to flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/CounterWrapper.java diff --git a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java similarity index 85% rename from flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java rename to flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java index d47090d8b36d5..fcb629a0f73c8 100644 --- a/flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java +++ b/flink-metric-reporters/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/GaugeWrapper.java @@ -19,15 +19,15 @@ import org.apache.flink.metrics.Gauge; -public class GaugeWrapper implements com.codahale.metrics.Gauge { - private final Gauge gauge; +public class GaugeWrapper implements com.codahale.metrics.Gauge { + private final Gauge gauge; - public GaugeWrapper(Gauge gauge) { + public GaugeWrapper(Gauge gauge) { this.gauge = gauge; } @Override - public Object getValue() { + public T getValue() { return this.gauge.getValue(); } } diff --git a/flink-metrics/flink-metrics-ganglia/pom.xml b/flink-metric-reporters/flink-metrics-ganglia/pom.xml similarity index 98% rename from flink-metrics/flink-metrics-ganglia/pom.xml rename to flink-metric-reporters/flink-metrics-ganglia/pom.xml index c4f51da9a2cec..a457ca1464b7c 100644 --- a/flink-metrics/flink-metrics-ganglia/pom.xml +++ b/flink-metric-reporters/flink-metrics-ganglia/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink - flink-metrics + flink-metric-reporters 1.1-SNAPSHOT .. diff --git a/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java b/flink-metric-reporters/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java similarity index 100% rename from flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java rename to flink-metric-reporters/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/graphite/GangliaReporter.java diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metric-reporters/flink-metrics-graphite/pom.xml similarity index 97% rename from flink-metrics/flink-metrics-graphite/pom.xml rename to flink-metric-reporters/flink-metrics-graphite/pom.xml index 45fb01857df4c..714b77f4327a1 100644 --- a/flink-metrics/flink-metrics-graphite/pom.xml +++ b/flink-metric-reporters/flink-metrics-graphite/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink - flink-metrics + flink-metric-reporters 1.1-SNAPSHOT .. diff --git a/flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java b/flink-metric-reporters/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java similarity index 100% rename from flink-metrics/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java rename to flink-metric-reporters/flink-metrics-graphite/src/main/java/org/apache/flink/metrics/graphite/GraphiteReporter.java diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metric-reporters/flink-metrics-statsd/pom.xml similarity index 96% rename from flink-metrics/flink-metrics-statsd/pom.xml rename to flink-metric-reporters/flink-metrics-statsd/pom.xml index 5d639087067d8..3052a1008f1e9 100644 --- a/flink-metrics/flink-metrics-statsd/pom.xml +++ b/flink-metric-reporters/flink-metrics-statsd/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink - flink-metrics + flink-metric-reporters 1.1-SNAPSHOT .. diff --git a/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java b/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java similarity index 98% rename from flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java rename to flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java index 288b8b86aa99b..e57001fa26f63 100644 --- a/flink-metrics/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java +++ b/flink-metric-reporters/flink-metrics-statsd/src/main/java/org/apache/flink/metrics/statsd/StatsDReporter.java @@ -110,7 +110,7 @@ public void send(final String name, final String value) { @Override public void report() { - for (Map.Entry entry : gauges.entrySet()) { + for (Map.Entry> entry : gauges.entrySet()) { reportGauge(entry.getKey(), entry.getValue()); } diff --git a/flink-metrics/pom.xml b/flink-metric-reporters/pom.xml similarity index 94% rename from flink-metrics/pom.xml rename to flink-metric-reporters/pom.xml index 542f49c26bc32..01a809ccba964 100644 --- a/flink-metrics/pom.xml +++ b/flink-metric-reporters/pom.xml @@ -29,8 +29,8 @@ under the License. .. - flink-metrics - flink-metrics + flink-metric-reporters + flink-metric-reporters pom diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index bc3ba0d9bc906..1f766e187e2a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -60,6 +60,7 @@ import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateUtils; import org.apache.flink.util.SerializedValue; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,15 +92,15 @@ * {@link org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable} have only data * readers, -writers, and certain event callbacks. The task connects those to the * network stack and actor messages, and tracks the state of the execution and - * handles exceptions.

+ * handles exceptions. * *

Tasks have no knowledge about how they relate to other tasks, or whether they * are the first attempt to execute the task, or a repeated attempt. All of that * is only known to the JobManager. All the task knows are its own runnable code, * the task's configuration, and the IDs of the intermediate results to consume and - * produce (if any).

+ * produce (if any). * - *

Each Task is run by one dedicated thread.

+ *

Each Task is run by one dedicated thread. */ public class Task implements Runnable { @@ -129,6 +130,7 @@ public class Task implements Runnable { /** TaskInfo object for this task */ private final TaskInfo taskInfo; + /** The name of the task, including subtask indexes */ private final String taskNameWithSubtask; /** The job-wide configuration object */ @@ -158,6 +160,9 @@ public class Task implements Runnable { /** The BroadcastVariableManager to be used by this task */ private final BroadcastVariableManager broadcastVariableManager; + /** Serialized version of the job specific execution configuration (see {@link ExecutionConfig}). */ + private final SerializedValue serializedExecutionConfig; + private final ResultPartition[] producedPartitions; private final ResultPartitionWriter[] writers; @@ -193,6 +198,9 @@ public class Task implements Runnable { /** The thread that executes the task */ private final Thread executingThread; + /** Parent group for all metrics of this task */ + private final TaskMetricGroup metrics; + // ------------------------------------------------------------------------ // Fields that control the task execution. All these fields are volatile // (which means that they introduce memory barriers), to establish @@ -202,8 +210,6 @@ public class Task implements Runnable { /** atomic flag that makes sure the invokable is canceled exactly once upon error */ private final AtomicBoolean invokableHasBeenCanceled; - private final TaskMetricGroup metrics; - /** The invokable of this task, if initialized */ private volatile AbstractInvokable invokable; @@ -215,16 +221,13 @@ public class Task implements Runnable { /** Serial executor for asynchronous calls (checkpoints, etc), lazily initialized */ private volatile ExecutorService asyncCallDispatcher; - + /** The handle to the state that the operator was initialized with. Will be set to null after the * initialization, to be memory friendly */ private volatile SerializedValue> operatorState; private volatile long recoveryTs; - /** Serialized version of the job specific execution configuration (see {@link ExecutionConfig}). */ - private final SerializedValue serializedExecutionConfig; - /** Initialized from the Flink configuration. May also be set at the ExecutionConfig */ private long taskCancellationInterval; @@ -688,9 +691,6 @@ else if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) { // remove all of the tasks library resources libraryCache.unregisterTask(jobId, executionId); - - //Uncomment before Merging!!! - //metrics.close(); // remove all files in the distributed cache removeCachedFiles(distributedCacheEntries, fileCache); @@ -703,6 +703,16 @@ else if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) { LOG.error(message, t); notifyFatalError(message, t); } + + // un-register the metrics at the end so that the task may already be + // counted as finished when this happens + // errors here will only be logged + try { + metrics.close(); + } + catch (Throwable t) { + LOG.error("Error during metrics de-registration", t); + } } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index beb012c9bed1f..a5cc18d5622cd 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -155,8 +155,8 @@ class TaskManager( /** Registry of metrics periodically transmitted to the JobManager */ private val metricRegistry = TaskManager.createMetricsRegistry() - private var metricsRegistry : FlinkMetricRegistry = null - private var taskManagerMetricGroup : TaskManagerMetricGroup = null + private var metricsRegistry : FlinkMetricRegistry = _ + private var taskManagerMetricGroup : TaskManagerMetricGroup = _ /** Metric serialization */ private val metricRegistryMapper: ObjectMapper = new ObjectMapper() @@ -938,7 +938,7 @@ class TaskManager( libraryCacheManager = Some(new FallbackLibraryCacheManager) } - metricsRegistry = new FlinkMetricRegistry(this.config.configuration); + metricsRegistry = new FlinkMetricRegistry(config.configuration) taskManagerMetricGroup = new TaskManagerMetricGroup(metricsRegistry, this.runtimeInfo.getHostname, id.toString) @@ -1011,6 +1011,10 @@ class TaskManager( // disassociate the network environment network.disassociate() + + // stop the metrics reporters + metricsRegistry.shutdown() + metricsRegistry = null } protected def handleJobManagerDisconnect(jobManager: ActorRef, msg: String): Unit = { @@ -1085,8 +1089,9 @@ class TaskManager( } val taskMetricGroup = taskManagerMetricGroup - .addJob(tdd.getJobID, jobName) - .addTask(tdd.getVertexID, tdd.getExecutionId, tdd.getIndexInSubtaskGroup, tdd.getTaskName) + .addTaskForJob( + tdd.getJobID, jobName, + tdd.getVertexID, tdd.getExecutionId, tdd.getIndexInSubtaskGroup, tdd.getTaskName) val task = new Task( tdd, @@ -1224,16 +1229,16 @@ class TaskManager( registry.getSnapshot } - self ! decorateMessage( - UpdateTaskExecutionState( - new TaskExecutionState( - task.getJobID, - task.getExecutionId, - task.getExecutionState, - task.getFailureCause, - accumulators) - ) + self ! decorateMessage( + UpdateTaskExecutionState( + new TaskExecutionState( + task.getJobID, + task.getExecutionId, + task.getExecutionState, + task.getFailureCause, + accumulators) ) + ) } else { log.error(s"Cannot find task with ID $executionID to unregister.") diff --git a/pom.xml b/pom.xml index c97815af63576..784aa4091d7e4 100644 --- a/pom.xml +++ b/pom.xml @@ -74,7 +74,7 @@ under the License. flink-quickstart flink-contrib flink-dist - flink-metrics + flink-metric-reporters From d2df6402e186f095fc4f14995fffc32f580209ca Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 20 May 2016 14:29:12 +0200 Subject: [PATCH 48/70] [FLINK-3927][yarn] make container id consistent across Hadoop versions - introduce a unique container id independent of the Hadoop version - improve printing of exceptions during registration - minor improvements to the Yarn ResourceManager code This closes #2013 --- .../FlinkResourceManager.java | 4 +-- .../clusterframework/types/ResourceID.java | 14 ++++++++--- .../flink/runtime/jobmanager/JobManager.scala | 4 +-- .../flink/yarn/RegisteredYarnWorkerNode.java | 13 +++++----- .../flink/yarn/YarnContainerInLaunch.java | 16 ++++++++---- .../flink/yarn/YarnFlinkResourceManager.java | 25 ++++++++++++------- .../flink/yarn/YarnTaskManagerRunner.java | 5 ++-- 7 files changed, 51 insertions(+), 30 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java index a5c354c8676a1..8766e15d75733 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java @@ -353,7 +353,7 @@ private void handleRegisterResource(ActorRef jobManager, ActorRef taskManager, ResourceID resourceID = msg.resourceId(); try { Preconditions.checkNotNull(resourceID); - WorkerType newWorker = workerRegistered(msg.resourceId()); + WorkerType newWorker = workerRegistered(resourceID); WorkerType oldWorker = registeredWorkers.put(resourceID, newWorker); if (oldWorker != null) { LOG.warn("Worker {} had been registered before.", resourceID); @@ -363,7 +363,7 @@ private void handleRegisterResource(ActorRef jobManager, ActorRef taskManager, self()); } catch (Exception e) { // This may happen on duplicate task manager registration message to the job manager - LOG.warn("TaskManager resource registration failed for {}", resourceID); + LOG.warn("TaskManager resource registration failed for {}", resourceID, e); // tell the JobManager about the failure String eStr = ExceptionUtils.stringifyException(e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java index 8e482444fb8dc..e599456bcbe09 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.clusterframework.types; import org.apache.flink.util.AbstractID; +import org.apache.flink.util.Preconditions; import java.io.Serializable; @@ -32,10 +33,15 @@ public class ResourceID implements Serializable { private final String resourceId; public ResourceID(String resourceId) { + Preconditions.checkNotNull(resourceId, "ResourceID must not be null"); this.resourceId = resourceId; } - public String getResourceId() { + /** + * Gets the Resource Id as string + * @return Stringified version of the ResourceID + */ + public final String getResourceIdString() { return resourceId; } @@ -48,10 +54,10 @@ public static ResourceID generate() { } @Override - public boolean equals(Object o) { + public final boolean equals(Object o) { if (this == o) { return true; - } else if (o == null || getClass() != o.getClass()) { + } else if (o == null || !(o instanceof ResourceID)) { return false; } else { return resourceId.equals(((ResourceID) o).resourceId); @@ -59,7 +65,7 @@ public boolean equals(Object o) { } @Override - public int hashCode() { + public final int hashCode() { return resourceId.hashCode(); } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 3c633f3ad8b1d..540957d8dcd86 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -440,8 +440,8 @@ class JobManager( val taskManager = msg.getTaskManager val resourceId = msg.getResourceID - log.warn(s"TaskManager's resource id $resourceId is not registered with ResourceManager. " + - s"Refusing registration.") + log.warn(s"TaskManager's resource id $resourceId failed to register at ResourceManager. " + + s"Refusing registration because of\n${msg.getMessage}.") taskManager ! decorateMessage( RefuseRegistration(new IllegalStateException( diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/RegisteredYarnWorkerNode.java b/flink-yarn/src/main/java/org/apache/flink/yarn/RegisteredYarnWorkerNode.java index 1fdb32cc91797..a6c094d4b7a55 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/RegisteredYarnWorkerNode.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/RegisteredYarnWorkerNode.java @@ -24,22 +24,23 @@ import static java.util.Objects.requireNonNull; +/** + * A representation of a registered Yarn container managed by the {@link YarnFlinkResourceManager}. + */ public class RegisteredYarnWorkerNode extends ResourceID { - + /** The container on which the worker runs */ private final Container yarnContainer; - public RegisteredYarnWorkerNode( - ResourceID resourceId, Container yarnContainer) - { - super(resourceId.getResourceId()); + public RegisteredYarnWorkerNode(Container yarnContainer) { + super(yarnContainer.getId().toString()); this.yarnContainer = requireNonNull(yarnContainer); } public Container yarnContainer() { return yarnContainer; } - + // ------------------------------------------------------------------------ @Override diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnContainerInLaunch.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnContainerInLaunch.java index 87020dbf0355f..03c5b3a245d10 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnContainerInLaunch.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnContainerInLaunch.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.hadoop.yarn.api.records.Container; import static java.util.Objects.requireNonNull; @@ -26,17 +27,22 @@ * This class describes a container in which a TaskManager is being launched (or * has been launched) but where the TaskManager has not properly registered, yet. */ -public class YarnContainerInLaunch { - +public class YarnContainerInLaunch extends ResourceID { + private final Container container; - + private final long timestamp; - + + public YarnContainerInLaunch(Container container) { + this(container, System.currentTimeMillis()); + } + public YarnContainerInLaunch(Container container, long timestamp) { + super(container.getId().toString()); this.container = requireNonNull(container); this.timestamp = timestamp; } - + // ------------------------------------------------------------------------ public Container container() { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java index 4a21d5c4dd2c9..71fc371be24cd 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java @@ -73,6 +73,10 @@ public class YarnFlinkResourceManager extends FlinkResourceManager containersInLaunch; @@ -210,7 +214,8 @@ protected void initialize() throws Exception { final long now = System.currentTimeMillis(); for (Container c : containersFromPreviousAttempts) { - containersInLaunch.put(new ResourceID(c.getId().toString()), new YarnContainerInLaunch(c, now)); + YarnContainerInLaunch containerInLaunch = new YarnContainerInLaunch(c, now); + containersInLaunch.put(containerInLaunch, containerInLaunch); } // adjust the progress indicator @@ -332,7 +337,7 @@ protected RegisteredYarnWorkerNode workerRegistered(ResourceID resourceID) throw if (inLaunch == null) { throw new Exception("Cannot register Worker - unknown resource id " + resourceID); } else { - return new RegisteredYarnWorkerNode(resourceID, inLaunch.container()); + return new RegisteredYarnWorkerNode(inLaunch.container()); } } @@ -346,7 +351,7 @@ protected Collection reacceptRegisteredWorkers(Collect if (yci != null) { LOG.info("YARN container consolidation recognizes Resource {} ", resourceID); - accepted.add(new RegisteredYarnWorkerNode(resourceID, yci.container())); + accepted.add(new RegisteredYarnWorkerNode(yci.container())); } else { LOG.info("YARN container consolidation does not recognize TaskManager {}", @@ -382,24 +387,26 @@ private void containersAllocated(List containers) { // decide whether to return the container, or whether to start a TaskManager if (numRegistered + containersInLaunch.size() < numRequired) { // start a TaskManager - final ResourceID containerIdString = new ResourceID(container.getId().toString()); - final long now = System.currentTimeMillis(); - containersInLaunch.put(containerIdString, new YarnContainerInLaunch(container, now)); + final YarnContainerInLaunch containerInLaunch = new YarnContainerInLaunch(container); + containersInLaunch.put(containerInLaunch, containerInLaunch); - String message = "Launching TaskManager in container " + containerIdString + String message = "Launching TaskManager in container " + containerInLaunch + " on host " + container.getNodeId().getHost(); LOG.info(message); sendInfoMessage(message); try { + // set a special environment variable to uniquely identify this container + taskManagerLaunchContext.getEnvironment() + .put(ENV_FLINK_CONTAINER_ID, containerInLaunch.getResourceIdString()); nodeManagerClient.startContainer(container, taskManagerLaunchContext); } catch (Throwable t) { // failed to launch the container - containersInLaunch.remove(containerIdString); + containersInLaunch.remove(containerInLaunch); // return container, a new one will be requested eventually - LOG.error("Could not start TaskManager in container " + containerIdString, t); + LOG.error("Could not start TaskManager in container " + containerInLaunch, t); containersBeingReturned.put(container.getId(), container); resourceManagerClient.releaseAssignedContainer(container.getId()); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java index dba81decb3b2b..6839bb56e5c7a 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.security.PrivilegedAction; import java.util.Map; -import java.util.Objects; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -29,6 +28,7 @@ import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; @@ -87,8 +87,9 @@ public static void runYarnTaskManager(String[] args, final Class() { @Override From 08c773e3753f702a49b4caefa2ecae4f8c418582 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 23 May 2016 12:06:25 +0200 Subject: [PATCH 49/70] [FLINK-3953][maven] rename unit-tests execution to default-test After 38698c0b101cbb48f8c10adf4060983ac07e2f4b, there are now two executions defined for the Surefire plugin: unit-tests and integration-tests. In addition, there is an implicit default execution called default-test. This leads to the unit tests to be executed twice. This renames unit-tests to default-test to prevent duplicate execution. This closes #2019 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 784aa4091d7e4..22515d009ab4a 100644 --- a/pom.xml +++ b/pom.xml @@ -923,7 +923,7 @@ under the License. - unit-tests + default-test test test From ccb91f2e170ebe784f4468cd6e12e3c10225a5ba Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 23 May 2016 14:02:21 +0200 Subject: [PATCH 50/70] [FLINK-3939] [tableAPI] Prevent translation of unsupported distinct aggregates and grouping sets. This closes #2014 --- .../api/table/BatchTableEnvironment.scala | 5 ++ .../rules/dataSet/DataSetAggregateRule.scala | 49 +++++++++++++------ .../scala/batch/sql/AggregationsITCase.scala | 46 +++++++++++++++++ 3 files changed, 85 insertions(+), 15 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala index 207500a723472..b25c940fc6097 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala @@ -256,6 +256,11 @@ abstract class BatchTableEnvironment( s"Cannot generate a valid execution plan for the given query: \n\n" + s"${RelOptUtil.toString(relNode)}\n" + "Please consider filing a bug report.", e) + case t: TableException => + throw new PlanGenException( + s"Cannot generate a valid execution plan for the given query: \n\n" + + s"${RelOptUtil.toString(relNode)}\n" + + t.msg) case a: AssertionError => throw a.getCause } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala index 0449fc33435f7..9f78adb727ae1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala @@ -18,10 +18,11 @@ package org.apache.flink.api.table.plan.rules.dataSet -import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet} +import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.logical.LogicalAggregate +import org.apache.flink.api.table.TableException import org.apache.flink.api.table.plan.nodes.dataset.{DataSetAggregate, DataSetConvention} import scala.collection.JavaConversions._ @@ -33,20 +34,38 @@ class DataSetAggregateRule "DataSetAggregateRule") { - def convert(rel: RelNode): RelNode = { - val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate] - val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE) - val convInput: RelNode = RelOptRule.convert(agg.getInput, DataSetConvention.INSTANCE) - - new DataSetAggregate( - rel.getCluster, - traitSet, - convInput, - agg.getNamedAggCalls, - rel.getRowType, - agg.getInput.getRowType, - agg.getGroupSet.toArray) - } + override def matches(call: RelOptRuleCall): Boolean = { + val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate] + + // check if we have distinct aggregates + val distinctAggs = agg.getAggCallList.exists(_.isDistinct) + if (distinctAggs) { + throw new TableException("DISTINCT aggregates are currently not supported.") + } + + // check if we have grouping sets + val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet + if (groupSets || agg.indicator) { + throw new TableException("GROUPING SETS are currently not supported.") + } + + !distinctAggs && !groupSets && !agg.indicator + } + + override def convert(rel: RelNode): RelNode = { + val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate] + val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE) + val convInput: RelNode = RelOptRule.convert(agg.getInput, DataSetConvention.INSTANCE) + + new DataSetAggregate( + rel.getCluster, + traitSet, + convInput, + agg.getNamedAggCalls, + rel.getRowType, + agg.getInput.getRowType, + agg.getGroupSet.toArray) + } } object DataSetAggregateRule { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala index 49541eb2b057a..73c0724247471 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala @@ -23,6 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import TableProgramsTestBase.TableConfigMode +import org.apache.flink.api.table.plan.PlanGenException import org.apache.flink.api.table.{Row, TableEnvironment} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils @@ -211,4 +212,49 @@ class AggregationsITCase( val results = result.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } + + @Test(expected = classOf[PlanGenException]) + def testDistinctAggregate(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT sum(_1) as a, count(distinct _3) as b FROM MyTable" + + val ds = CollectionDataSets.get3TupleDataSet(env) + tEnv.registerDataSet("MyTable", ds) + + // must fail. distinct aggregates are not supported + tEnv.sql(sqlQuery).toDataSet[Row] + } + + @Test(expected = classOf[PlanGenException]) + def testGroupedDistinctAggregate(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT _2, avg(distinct _1) as a, count(_3) as b FROM MyTable GROUP BY _2" + + val ds = CollectionDataSets.get3TupleDataSet(env) + tEnv.registerDataSet("MyTable", ds) + + // must fail. distinct aggregates are not supported + tEnv.sql(sqlQuery).toDataSet[Row] + } + + @Test(expected = classOf[PlanGenException]) + def testGroupingSetAggregate(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT _2, _3, avg(_1) as a FROM MyTable GROUP BY GROUPING SETS (_2, _3)" + + val ds = CollectionDataSets.get3TupleDataSet(env) + tEnv.registerDataSet("MyTable", ds) + + // must fail. grouping sets are not supported + tEnv.sql(sqlQuery).toDataSet[Row] + } } From ef667cf32f916b80fdab687c26c047cab0e8cf77 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 23 May 2016 14:03:16 +0200 Subject: [PATCH 51/70] [hotfix] [tableAPI] Throw helpful exception for unsupported ORDER BY features. --- .../plan/rules/dataSet/DataSetSortRule.scala | 10 ++++++++ .../api/scala/batch/sql/SortITCase.scala | 25 +++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala index b7f70e33f33af..b26d1decd173a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala @@ -23,6 +23,7 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rel.logical.{LogicalJoin, LogicalSort} +import org.apache.flink.api.table.TableException import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetSort} class DataSetSortRule @@ -37,6 +38,15 @@ class DataSetSortRule */ override def matches(call: RelOptRuleCall): Boolean = { val sort = call.rel(0).asInstanceOf[LogicalSort] + + if (sort.offset != null) { + throw new TableException("ORDER BY OFFSET is currently not supported.") + } + + if (sort.fetch != null) { + throw new TableException("ORDER BY FETCH is currently not supported.") + } + sort.offset == null && sort.fetch == null } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala index 0dea0b6aaa62b..7206be719be00 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala @@ -25,6 +25,7 @@ import org.apache.flink.api.scala.batch.utils.SortTestUtils._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala._ +import org.apache.flink.api.table.plan.PlanGenException import org.apache.flink.api.table.{Row, TableEnvironment} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils @@ -60,4 +61,28 @@ class SortITCase( TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } + @Test(expected = classOf[PlanGenException]) + def testOrderByOffset(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 OFFSET 2 ROWS" + + val ds = CollectionDataSets.get3TupleDataSet(env) + tEnv.registerDataSet("MyTable", ds) + tEnv.sql(sqlQuery).toDataSet[Row] + } + + @Test(expected = classOf[PlanGenException]) + def testOrderByFirst(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 FETCH NEXT 2 ROWS ONLY" + + val ds = CollectionDataSets.get3TupleDataSet(env) + tEnv.registerDataSet("MyTable", ds) + tEnv.sql(sqlQuery).toDataSet[Row] + } + } From 0f0869cdc9d886a8727f94999e837787565ddda1 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 23 May 2016 14:05:09 +0200 Subject: [PATCH 52/70] [hotfix] [tableAPI] Throw helpful exception for unsupported outer joins. --- .../flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala index f3bd402fd01ac..a4aa0a8746a91 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala @@ -24,6 +24,7 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rel.logical.LogicalJoin import org.apache.flink.api.java.operators.join.JoinType +import org.apache.flink.api.table.TableException import org.apache.flink.api.table.plan.nodes.dataset.{DataSetJoin, DataSetConvention} import scala.collection.JavaConversions._ @@ -44,6 +45,9 @@ class DataSetJoinRule val hasValidCondition = !joinInfo.pairs().isEmpty // only inner joins are supported at the moment val isInnerJoin = join.getJoinType.equals(JoinRelType.INNER) + if (!isInnerJoin) { + throw new TableException("OUTER JOIN is currently not supported.") + } // check that condition is valid and inner join hasValidCondition && isInnerJoin From e39f5dc9ff4ad9edb5676a31b13c219a20bf8f07 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Wed, 18 May 2016 23:57:42 +0800 Subject: [PATCH 53/70] [FLINK-3632] [tableAPI] Clean up TableAPI exceptions. This closes #2015 --- .../api/scala/table/TableConversions.scala | 2 +- .../flink/api/table/TableEnvironment.scala | 29 +++++++++---------- .../api/table/plan/logical/operators.scala | 3 +- .../table/plan/nodes/dataset/DataSetRel.scala | 4 +-- .../api/table/plan/rules/FlinkRuleSets.scala | 2 +- .../api/table/plan/schema/FlinkTable.scala | 12 ++++---- .../api/table/sources/CsvTableSource.scala | 10 +++---- .../flink/api/table/trees/TreeNode.scala | 5 ++-- .../api/table/typeutils/RowTypeInfo.scala | 6 ++-- .../java/batch/table/FromDataSetITCase.java | 11 +++---- .../api/scala/batch/table/ToTableITCase.scala | 12 ++++---- .../api/scala/stream/table/SelectITCase.scala | 12 ++++---- 12 files changed, 54 insertions(+), 54 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala index 1fdcbc59fd2e3..720dac02709f8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala @@ -41,7 +41,7 @@ class TableConversions(table: Table) { tEnv.toDataSet(table) case _ => throw new TableException( - "Only tables that orginate from Scala DataSets can be converted to Scala DataSets.") + "Only tables that originate from Scala DataSets can be converted to Scala DataSets.") } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala index 8aa9e1063f8e2..1c592f92e2184 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala @@ -236,8 +236,7 @@ abstract class TableEnvironment(val config: TableConfig) { case c: CaseClassTypeInfo[A] => c.getFieldNames case p: PojoTypeInfo[A] => p.getFieldNames case tpe => - throw new IllegalArgumentException( - s"Type $tpe requires explicit field naming.") + throw new TableException(s"Type $tpe lacks explicit field naming") } val fieldIndexes = fieldNames.indices.toArray (fieldNames, fieldIndexes) @@ -259,12 +258,11 @@ abstract class TableEnvironment(val config: TableConfig) { val indexedNames: Array[(Int, String)] = inputType match { case a: AtomicType[A] => if (exprs.length != 1) { - throw new IllegalArgumentException("Atomic type may can only have a single field.") + throw new TableException("Table of atomic type can only have a single field.") } exprs.map { case UnresolvedFieldReference(name) => (0, name) - case _ => throw new IllegalArgumentException( - "Field reference expression expected.") + case _ => throw new TableException("Field reference expression expected.") } case t: TupleTypeInfo[A] => exprs.zipWithIndex.map { @@ -272,11 +270,11 @@ abstract class TableEnvironment(val config: TableConfig) { case (Alias(UnresolvedFieldReference(origName), name), _) => val idx = t.getFieldIndex(origName) if (idx < 0) { - throw new IllegalArgumentException(s"$origName is not a field of type $t") + throw new TableException(s"$origName is not a field of type $t") } (idx, name) - case _ => throw new IllegalArgumentException( - "Field reference expression or naming expression expected.") + case _ => throw new TableException( + "Field reference expression or alias on field expression expected.") } case c: CaseClassTypeInfo[A] => exprs.zipWithIndex.map { @@ -284,25 +282,24 @@ abstract class TableEnvironment(val config: TableConfig) { case (Alias(UnresolvedFieldReference(origName), name), _) => val idx = c.getFieldIndex(origName) if (idx < 0) { - throw new IllegalArgumentException(s"$origName is not a field of type $c") + throw new TableException(s"$origName is not a field of type $c") } (idx, name) - case _ => throw new IllegalArgumentException( - "Field reference expression or naming expression expected.") + case _ => throw new TableException( + "Field reference expression or alias on field expression expected.") } case p: PojoTypeInfo[A] => exprs.map { case Alias(UnresolvedFieldReference(origName), name) => val idx = p.getFieldIndex(origName) if (idx < 0) { - throw new IllegalArgumentException(s"$origName is not a field of type $p") + throw new TableException(s"$origName is not a field of type $p") } (idx, name) - case _ => throw new IllegalArgumentException( - "Field naming expression expected.") + case _ => throw new TableException("Alias on field reference expression expected.") } - case tpe => throw new IllegalArgumentException( - s"Type $tpe cannot be converted into Table.") + case tpe => throw new TableException( + s"Source of type $tpe cannot be converted into Table.") } val (fieldIndexes, fieldNames) = indexedNames.unzip diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala index d3476510e9253..bd299b36ab94e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala @@ -46,7 +46,8 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extend case c @ Cast(ne: NamedExpression, tp) => Alias(c, s"${ne.name}-$tp") case other => Alias(other, s"_c$i") } - case _ => throw new IllegalArgumentException + case _ => + throw new RuntimeException("This should never be called and probably points to a bug.") } } Project(newProjectList, child) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala index 8c14e9e56238c..946dfc0f6c6dc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala @@ -27,7 +27,7 @@ import org.apache.flink.api.java.DataSet import org.apache.flink.api.table.codegen.CodeGenerator import org.apache.flink.api.table.plan.nodes.FlinkRel import org.apache.flink.api.table.runtime.MapRunner -import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig} +import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig, TableException} import scala.collection.JavaConversions._ @@ -61,7 +61,7 @@ trait DataSetRel extends RelNode with FlinkRel { case SqlTypeName.DOUBLE => s + 8 case SqlTypeName.VARCHAR => s + 12 case SqlTypeName.CHAR => s + 1 - case _ => throw new IllegalArgumentException("Unsupported data type encountered") + case _ => throw new TableException("Unsupported data type encountered") } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala index 4ce8e5f165042..a2ec08d0c9880 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala @@ -108,7 +108,7 @@ object FlinkRuleSets { ) /** - * RuleSet to optimize plans for batch / DataSet execution + * RuleSet to optimize plans for stream / DataStream execution */ val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala index 9414fae38b7b0..7024ce2c89d38 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala @@ -23,8 +23,8 @@ import org.apache.calcite.schema.impl.AbstractTable import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.api.common.typeinfo.{TypeInformation, AtomicType} import org.apache.flink.api.common.typeutils.CompositeType +import org.apache.flink.api.table.TableException import org.apache.flink.api.table.typeutils.TypeConverter -import org.apache.flink.streaming.api.datastream.DataStream abstract class FlinkTable[T]( val typeInfo: TypeInformation[T], @@ -33,13 +33,13 @@ abstract class FlinkTable[T]( extends AbstractTable { if (fieldIndexes.length != fieldNames.length) { - throw new IllegalArgumentException( + throw new TableException( "Number of field indexes and field names must be equal.") } // check uniqueness of field names if (fieldNames.length != fieldNames.toSet.size) { - throw new IllegalArgumentException( + throw new TableException( "Table field names must be unique.") } @@ -47,8 +47,8 @@ abstract class FlinkTable[T]( typeInfo match { case cType: CompositeType[T] => if (fieldNames.length != cType.getArity) { - throw new IllegalArgumentException( - s"Arity of DataStream type (" + cType.getFieldNames.deep + ") " + + throw new TableException( + s"Arity of type (" + cType.getFieldNames.deep + ") " + "not equal to number of field names " + fieldNames.deep + ".") } fieldIndexes @@ -56,7 +56,7 @@ abstract class FlinkTable[T]( .map(TypeConverter.typeInfoToSqlType(_)) case aType: AtomicType[T] => if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) { - throw new IllegalArgumentException( + throw new TableException( "Non-composite input type may have only a single field and its index must be 0.") } Array(TypeConverter.typeInfoToSqlType(aType)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala index b9ccbf97db0ff..9296fe235705f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala @@ -21,9 +21,9 @@ package org.apache.flink.api.table.sources import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.io.TupleCsvInputFormat import org.apache.flink.api.java.tuple.Tuple -import org.apache.flink.api.java.typeutils.{TupleTypeInfoBase, TupleTypeInfo} -import org.apache.flink.api.java.{ExecutionEnvironment, DataSet} -import org.apache.flink.api.table.Row +import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TupleTypeInfoBase} +import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} +import org.apache.flink.api.table.{Row, TableException} import org.apache.flink.core.fs.Path /** @@ -52,11 +52,11 @@ class CsvTableSource( extends BatchTableSource[Tuple] { if (fieldNames.length != fieldTypes.length) { - throw new IllegalArgumentException("Number of field names and field types must be equal.") + throw new TableException("Number of field names and field types must be equal.") } if (fieldNames.length > 25) { - throw new IllegalArgumentException("Only up to 25 fields supported with this CsvTableSource.") + throw new TableException("Only up to 25 fields supported with this CsvTableSource.") } /** Returns the data of the table as a [[DataSet]] of [[Row]]. */ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala index 63c7013d11b39..5d3ffefb3a458 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala @@ -107,8 +107,9 @@ abstract class TreeNode[A <: TreeNode[A]] extends Product { self: A => try { defaultCtor.newInstance(newArgs: _*).asInstanceOf[A] } catch { - case e: java.lang.IllegalArgumentException => - throw new IllegalArgumentException(s"Fail to copy treeNode ${getClass.getName}") + case e: Throwable => + throw new RuntimeException( + s"Fail to copy treeNode ${getClass.getName}: ${e.getStackTraceString}") } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala index f0f169fd995d2..d606a7624d76f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala @@ -25,7 +25,7 @@ import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import scala.collection.mutable.ArrayBuffer import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.api.table.Row +import org.apache.flink.api.table.{Row, TableException} /** * TypeInformation for [[Row]]. @@ -39,10 +39,10 @@ class RowTypeInfo(fieldTypes: Seq[TypeInformation[_]], fieldNames: Seq[String]) { if (fieldTypes.length != fieldNames.length) { - throw new IllegalArgumentException("Number of field types and names is different.") + throw new TableException("Number of field types and names is different.") } if (fieldNames.length != fieldNames.toSet.size) { - throw new IllegalArgumentException("Field names are not unique.") + throw new TableException("Field names are not unique.") } def this(fieldTypes: Seq[TypeInformation[_]]) = { diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java index a3204f93d7b7e..ecd916f0c3d0c 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/FromDataSetITCase.java @@ -30,6 +30,7 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.table.TableEnvironment; +import org.apache.flink.api.table.TableException; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; import org.junit.Test; import org.junit.runner.RunWith; @@ -220,7 +221,7 @@ public void testAsFromAndToPrivateFieldPojo() throws Exception { compareResultAsText(results, expected); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = TableException.class) public void testAsWithToFewFields() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -229,7 +230,7 @@ public void testAsWithToFewFields() throws Exception { tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b"); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = TableException.class) public void testAsWithToManyFields() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -238,7 +239,7 @@ public void testAsWithToManyFields() throws Exception { tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c, d"); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = TableException.class) public void testAsWithAmbiguousFields() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -247,7 +248,7 @@ public void testAsWithAmbiguousFields() throws Exception { tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, b"); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = TableException.class) public void testAsWithNonFieldReference1() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); @@ -256,7 +257,7 @@ public void testAsWithNonFieldReference1() throws Exception { tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a + 1, b, c"); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = TableException.class) public void testAsWithNonFieldReference2() throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config()); diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala index ed7b88f3f4135..84bdbb0e9ec36 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/ToTableITCase.scala @@ -23,7 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, TableException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -101,7 +101,7 @@ class ToTableITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testToTableWithToFewFields(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -111,7 +111,7 @@ class ToTableITCase( .toTable(tEnv, 'a, 'b) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testToTableWithToManyFields(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -121,7 +121,7 @@ class ToTableITCase( .toTable(tEnv, 'a, 'b, 'c, 'd) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testToTableWithAmbiguousFields(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -131,7 +131,7 @@ class ToTableITCase( .toTable(tEnv, 'a, 'b, 'b) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testToTableWithNonFieldReference1(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -141,7 +141,7 @@ class ToTableITCase( .toTable(tEnv, 'a + 1, 'b, 'c) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testToTableWithNonFieldReference2(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala index e3eb19a7c5b21..c6a2139d554d6 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/SelectITCase.scala @@ -19,9 +19,9 @@ package org.apache.flink.api.scala.stream.table import org.apache.flink.api.scala._ -import org.apache.flink.api.scala.stream.utils.{StreamTestData, StreamITCase} +import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData} import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{Row, TableEnvironment, TableException} import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.junit.Assert._ @@ -108,7 +108,7 @@ class SelectITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testAsWithToFewFields(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -124,7 +124,7 @@ class SelectITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testAsWithToManyFields(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -140,7 +140,7 @@ class SelectITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testAsWithAmbiguousFields(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -157,7 +157,7 @@ class SelectITCase extends StreamingMultipleProgramsTestBase { } - @Test(expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[TableException]) def testOnlyFieldRefInAs(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment From 7650ba4330d0bd3e2286dea251ee35e97860c6b0 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Mon, 23 May 2016 17:25:57 +0200 Subject: [PATCH 54/70] [docs] Fix outdated default value for akka.ask.timeout --- docs/setup/config.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/setup/config.md b/docs/setup/config.md index 14e9d21a405b6..b4d0242424619 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -175,7 +175,7 @@ The following parameters configure Flink's JobManager and TaskManagers. ### Distributed Coordination (via Akka) -- `akka.ask.timeout`: Timeout used for all futures and blocking Akka calls. If Flink fails due to timeouts then you should try to increase this value. Timeouts can be caused by slow machines or a congested network. The timeout value requires a time-unit specifier (ms/s/min/h/d) (DEFAULT: **100 s**). +- `akka.ask.timeout`: Timeout used for all futures and blocking Akka calls. If Flink fails due to timeouts then you should try to increase this value. Timeouts can be caused by slow machines or a congested network. The timeout value requires a time-unit specifier (ms/s/min/h/d) (DEFAULT: **10 s**). - `akka.lookup.timeout`: Timeout used for the lookup of the JobManager. The timeout value has to contain a time-unit specifier (ms/s/min/h/d) (DEFAULT: **10 s**). - `akka.framesize`: Maximum size of messages which are sent between the JobManager and the TaskManagers. If Flink fails because messages exceed this limit, then you should increase it. The message size requires a size-unit specifier (DEFAULT: **10485760b**). - `akka.watch.heartbeat.interval`: Heartbeat interval for Akka's DeathWatch mechanism to detect dead TaskManagers. If TaskManagers are wrongly marked dead because of lost or delayed heartbeat messages, then you should increase this value. A thorough description of Akka's DeathWatch can be found [here](http://doc.akka.io/docs/akka/snapshot/scala/remoting.html#failure-detector) (DEFAULT: **akka.ask.timeout/10**). From 79166ea5a31b9c1997d2a3125ba1d58fcebbb33b Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Sun, 22 May 2016 19:32:01 +0200 Subject: [PATCH 55/70] [hotfix] [tableAPI] Moved tests to correct package. --- .../expressions}/ScalarFunctionsTest.scala | 4 ++-- .../expressions}/utils/ExpressionEvaluator.scala | 2 +- .../api/{scala => table}/typeutils/RowComparatorTest.scala | 6 +++--- .../api/{scala => table}/typeutils/RowSerializerTest.scala | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/api/{scala/expression => table/expressions}/ScalarFunctionsTest.scala (98%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/api/{scala/expression => table/expressions}/utils/ExpressionEvaluator.scala (98%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/api/{scala => table}/typeutils/RowComparatorTest.scala (96%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/api/{scala => table}/typeutils/RowSerializerTest.scala (98%) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala similarity index 98% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/ScalarFunctionsTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala index 8d1cfa207aa6d..a3b67f511ec0c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.api.scala.expression +package org.apache.flink.api.table.expressions import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.expression.utils.ExpressionEvaluator +import org.apache.flink.api.table.expressions.utils.ExpressionEvaluator import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.Row import org.apache.flink.api.table.expressions.{Expression, ExpressionParser} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionEvaluator.scala similarity index 98% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionEvaluator.scala index fe606e0e0e23a..0b5a2dedd4267 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/expression/utils/ExpressionEvaluator.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionEvaluator.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.scala.expression.utils +package org.apache.flink.api.table.expressions.utils import org.apache.calcite.rel.logical.LogicalProject import org.apache.calcite.rex.RexNode diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/typeutils/RowComparatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala similarity index 96% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/typeutils/RowComparatorTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala index 9ceb9d299f0eb..557db3abba594 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/typeutils/RowComparatorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala @@ -16,16 +16,15 @@ * limitations under the License. */ -package org.apache.flink.api.scala.typeutils +package org.apache.flink.api.table.typeutils import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer} import org.apache.flink.api.java.tuple import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} -import org.apache.flink.api.scala.typeutils.RowComparatorTest.MyPojo import org.apache.flink.api.table.Row -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.table.typeutils.RowComparatorTest.MyPojo import org.junit.Assert._ class RowComparatorTest extends ComparatorTestBase[Row] { @@ -134,3 +133,4 @@ object RowComparatorTest { } } } + diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/typeutils/RowSerializerTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala similarity index 98% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/typeutils/RowSerializerTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala index 95a1bb5ceea65..b52dd4d63762c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/typeutils/RowSerializerTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.common.typeutils.{SerializerTestInstance, TypeSerializer} import org.apache.flink.api.java.tuple -import org.apache.flink.api.java.typeutils.{TypeExtractor, TupleTypeInfo} +import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} import org.apache.flink.api.table.Row import org.apache.flink.api.table.typeutils.RowSerializerTest.MyPojo import org.junit.Assert._ From 9d0fd5bb4b2f444659b839a97c3848c39df439bd Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 23 May 2016 16:49:52 +0200 Subject: [PATCH 56/70] [FLINK-3955] [tableAPI] Rename Table.toSink() to Table.writeToSink(). This closes #2023 --- docs/apis/table.md | 16 ++++++++-------- .../flink/api/table/BatchTableEnvironment.scala | 10 +++++----- .../flink/api/table/StreamTableEnvironment.scala | 10 +++++----- .../flink/api/table/TableEnvironment.scala | 8 ++++---- .../scala/org/apache/flink/api/table/table.scala | 10 +++++----- .../flink/api/scala/batch/TableSinkITCase.scala | 2 +- .../flink/api/scala/stream/TableSinkITCase.scala | 2 +- 7 files changed, 29 insertions(+), 29 deletions(-) diff --git a/docs/apis/table.md b/docs/apis/table.md index 276341db154df..f33ae5920e5a8 100644 --- a/docs/apis/table.md +++ b/docs/apis/table.md @@ -685,12 +685,12 @@ SQL queries can be executed on DataStream Tables by adding the `STREAM` SQL keyw {% top %} -Emit a Table to external sinks +Write Tables to external sinks ---- -A `Table` can be emitted to a `TableSink`, which is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ), and others. A batch `Table` can only be emitted by a `BatchTableSink`, a streaming table requires a `StreamTableSink` (a `TableSink` can implement both interfaces). +A `Table` can be written to a `TableSink`, which is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ). A batch `Table` can only be written to a `BatchTableSink`, a streaming table requires a `StreamTableSink`. A `TableSink` can implement both interfaces at the same time. -Currently, Flink only provides a `CsvTableSink` that writes a batch or streaming `Table` to CSV-formatted files. A custom `TableSource` can be defined by implementing the `BatchTableSink` and/or `StreamTableSink` interface. +Currently, Flink only provides a `CsvTableSink` that writes a batch or streaming `Table` to CSV-formatted files. A custom `TableSink` can be defined by implementing the `BatchTableSink` and/or `StreamTableSink` interface.

@@ -703,8 +703,8 @@ Table result = ... // create a TableSink TableSink sink = new CsvTableSink("/path/to/file", fieldDelim = "|"); -// add a TableSink to emit the result Table -result.toSink(sink); +// write the result Table to the TableSink +result.writeToSink(sink); // execute the program env.execute(); @@ -721,8 +721,8 @@ val result: Table = ... // create a TableSink val sink: TableSink = new CsvTableSink("/path/to/file", fieldDelim = "|") -// add a TableSink to emit the result Table -result.toSink(sink) +// write the result Table to the TableSink +result.writeToSink(sink) // execute the program env.execute() @@ -737,5 +737,5 @@ Runtime Configuration The Table API provides a configuration (the so-called `TableConfig`) to modify runtime behavior. It can be accessed either through `TableEnvironment` or passed to the `toDataSet`/`toDataStream` method when using Scala implicit conversion. ### Null Handling -By default, the Table API supports `null` values. Null handling can be disabled by setting the `nullCheck` property in the `TableConfig` to `false`. +By default, the Table API supports `null` values. Null handling can be disabled to improve preformance by setting the `nullCheck` property in the `TableConfig` to `false`. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala index b25c940fc6097..4c8b37091278e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala @@ -136,16 +136,16 @@ abstract class BatchTableEnvironment( } /** - * Emits a [[Table]] to a [[TableSink]]. + * Writes a [[Table]] to a [[TableSink]]. * * Internally, the [[Table]] is translated into a [[DataSet]] and handed over to the - * [[TableSink]] to emit it. + * [[TableSink]] to write it. * - * @param table The [[Table]] to emit. - * @param sink The [[TableSink]] to emit the [[Table]] to. + * @param table The [[Table]] to write. + * @param sink The [[TableSink]] to write the [[Table]] to. * @tparam T The expected type of the [[DataSet]] which represents the [[Table]]. */ - override private[flink] def emitToSink[T](table: Table, sink: TableSink[T]): Unit = { + override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = { sink match { case batchSink: BatchTableSink[T] => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala index 8ba30002f72e3..bacb587fa2052 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala @@ -137,16 +137,16 @@ abstract class StreamTableEnvironment( } /** - * Emits a [[Table]] to a [[TableSink]]. + * Writes a [[Table]] to a [[TableSink]]. * * Internally, the [[Table]] is translated into a [[DataStream]] and handed over to the - * [[TableSink]] to emit it. + * [[TableSink]] to write it. * - * @param table The [[Table]] to emit. - * @param sink The [[TableSink]] to emit the [[Table]] to. + * @param table The [[Table]] to write. + * @param sink The [[TableSink]] to write the [[Table]] to. * @tparam T The expected type of the [[DataStream]] which represents the [[Table]]. */ - override private[flink] def emitToSink[T](table: Table, sink: TableSink[T]): Unit = { + override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = { sink match { case streamSink: StreamTableSink[T] => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala index 1c592f92e2184..7debb65821352 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala @@ -147,13 +147,13 @@ abstract class TableEnvironment(val config: TableConfig) { def sql(query: String): Table /** - * Emits a [[Table]] to a [[TableSink]]. + * Writes a [[Table]] to a [[TableSink]]. * - * @param table The [[Table]] to emit. - * @param sink The [[TableSink]] to emit the [[Table]] to. + * @param table The [[Table]] to write. + * @param sink The [[TableSink]] to write the [[Table]] to. * @tparam T The data type that the [[TableSink]] expects. */ - private[flink] def emitToSink[T](table: Table, sink: TableSink[T]): Unit + private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit /** * Registers a Calcite [[AbstractTable]] in the TableEnvironment's catalog. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala index 4f111c9ac3529..1e558c5448c2d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala @@ -315,16 +315,16 @@ class Table( } /** - * Emits the [[Table]] to a [[TableSink]]. A [[TableSink]] defines an external storage location. + * Writes the [[Table]] to a [[TableSink]]. A [[TableSink]] defines an external storage location. * - * A batch [[Table]] can only be emitted by a + * A batch [[Table]] can only be written to a * [[org.apache.flink.api.table.sinks.BatchTableSink]], a streaming [[Table]] requires a * [[org.apache.flink.api.table.sinks.StreamTableSink]]. * - * @param sink The [[TableSink]] to which the [[Table]] is emitted. + * @param sink The [[TableSink]] to which the [[Table]] is written. * @tparam T The data type that the [[TableSink]] expects. */ - def toSink[T](sink: TableSink[T]): Unit = { + def writeToSink[T](sink: TableSink[T]): Unit = { // get schema information of table val rowType = getRelNode.getRowType @@ -336,7 +336,7 @@ class Table( val configuredSink = sink.configure(fieldNames, fieldTypes) // emit the table to the configured table sink - tableEnv.emitToSink(this, configuredSink) + tableEnv.writeToSink(this, configuredSink) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala index 39684ff914feb..dd0668c5108ed 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala @@ -59,7 +59,7 @@ class TableSinkITCase( val results = input.toTable(tEnv, 'a, 'b, 'c) .where('a < 5 || 'a > 17) .select('c, 'b) - .toSink(new CsvTableSink(path, fieldDelim = "|")) + .writeToSink(new CsvTableSink(path, fieldDelim = "|")) env.execute() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala index 66cb9bf407899..160d88a5fea1a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala @@ -53,7 +53,7 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase { val results = input.toTable(tEnv, 'a, 'b, 'c) .where('a < 5 || 'a > 17) .select('c, 'b) - .toSink(new CsvTableSink(path)) + .writeToSink(new CsvTableSink(path)) env.execute() From 14033d8d20f40d1ca4d968397ccd58da553625fd Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Sun, 22 May 2016 12:43:19 +0200 Subject: [PATCH 57/70] [FLINK-3728] [tableAPI] Improve error message and documentation for unsupported SQL features. This closes #2018 --- docs/apis/table.md | 99 ++++++++++++++----- .../api/table/BatchTableEnvironment.scala | 11 ++- .../api/table/StreamTableEnvironment.scala | 6 +- .../api/table/plan/PlanGenException.scala | 26 ----- .../runtime/aggregate/AggregateUtil.scala | 21 ++-- .../scala/batch/sql/AggregationsITCase.scala | 9 +- .../api/scala/batch/sql/JoinITCase.scala | 9 +- .../api/scala/batch/sql/SortITCase.scala | 7 +- .../api/scala/batch/table/JoinITCase.scala | 7 +- 9 files changed, 109 insertions(+), 86 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/PlanGenException.scala diff --git a/docs/apis/table.md b/docs/apis/table.md index f33ae5920e5a8..4e52a983500ed 100644 --- a/docs/apis/table.md +++ b/docs/apis/table.md @@ -31,7 +31,7 @@ under the License. The Table API is a SQL-like expression language for relational stream and batch processing that can be easily embedded in Flink's DataSet and DataStream APIs (Java and Scala). The Table API and SQL interface operate on a relational `Table` abstraction, which can be created from external data sources, or existing DataSets and DataStreams. With the Table API, you can apply relational operators such as selection, aggregation, and joins on `Table`s. -`Table`s can also be queried with regular SQL, as long as they are registered (see [Registering and Accessing Tables](#registering-and-accessing-tables)). The Table API and SQL offer equivalent functionality and can be mixed in the same program. When a `Table` is converted back into a `DataSet` or `DataStream`, the logical plan, which was defined by relational operators and SQL queries, is optimized using [Apache Calcite](https://calcite.apache.org/) and transformed into a `DataSet` or `DataStream` execution plan. +`Table`s can also be queried with regular SQL, as long as they are registered (see [Registering Tables](#registering-tables)). The Table API and SQL offer equivalent functionality and can be mixed in the same program. When a `Table` is converted back into a `DataSet` or `DataStream`, the logical plan, which was defined by relational operators and SQL queries, is optimized using [Apache Calcite](https://calcite.apache.org/) and transformed into a `DataSet` or `DataStream` program. * This will be replaced by the TOC {:toc} @@ -50,15 +50,15 @@ The following dependency must be added to your project in order to use the Table {% endhighlight %} -Note that the Table API is currently not part of the binary distribution. See linking with it for cluster execution [here]({{ site.baseurl }}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution). +*Note: The Table API is currently not part of the binary distribution. See linking with it for cluster execution [here]({{ site.baseurl }}/apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).* Registering Tables -------------------------------- -`TableEnvironment`s have an internal table catalog to which tables can be registered with a unique name. After registration, a table can be accessed from the `TableEnvironment` by its name. Tables can be registered in different ways. +`TableEnvironment`s have an internal table catalog to which tables can be registered with a unique name. After registration, a table can be accessed from the `TableEnvironment` by its name. -*Note that it is not required to register a `DataSet` or `DataStream` as a table in a `TableEnvironment` in order to process it with the Table API.* +*Note: `DataSet`s or `DataStream`s can be directly converted into `Table`s without registering them in the `TableEnvironment`.* ### Register a DataSet @@ -92,7 +92,7 @@ tableEnv.registerDataSet("Orders", ord, 'user, 'product, 'amount)
-*Note: DataSet table names are not allowed to follow the `^_DataSetTable_[0-9]+` pattern, as these are reserved for internal use only.* +*Note: The name of a `DataSet` `Table` must not match the `^_DataSetTable_[0-9]+` pattern which is reserved for internal use only.* ### Register a DataStream @@ -126,7 +126,7 @@ tableEnv.registerDataStream("Orders", ord, 'user, 'product, 'amount) -*Note: DataStream table names are not allowed to follow the `^_DataStreamTable_[0-9]+` pattern, as these are reserved for internal use only.* +*Note: The name of a `DataStream` `Table` must not match the `^_DataStreamTable_[0-9]+` pattern which is reserved for internal use only.* ### Register a Table @@ -214,7 +214,7 @@ The Table API provides methods to apply relational operations on DataSets and Da The central concept of the Table API is a `Table` which represents a table with relational schema (or relation). Tables can be created from a `DataSet` or `DataStream`, converted into a `DataSet` or `DataStream`, or registered in a table catalog using a `TableEnvironment`. A `Table` is always bound to a specific `TableEnvironment`. It is not possible to combine Tables of different TableEnvironments. -*Note that the only operations currently supported on streaming Tables are selection, filtering, and union.* +*Note: The only operations currently supported on streaming Tables are selection, projection, and union.*
@@ -639,17 +639,13 @@ column names and function names follow Java identifier syntax. Expressions speci SQL ---- -Registered `Table`s can be directly queried with SQL and SQL queries can also be mixed with Table API expressions. Table API and SQL statements will be translated into a single optimized DataStream or DataSet program. +SQL queries are specified using the `sql()` method of the `TableEnvironment`. The method returns the result of the SQL query as a `Table` which can be converted into a `DataSet` or `DataStream`, used in subsequent Table API queries, or written to a `TableSink` (see [Writing Tables to External Sinks](#writing-tables-to-external-sinks)). SQL and Table API queries can seamlessly mixed and are holistically optimized and translated into a single DataStream or DataSet program. -*Note: The current SQL implementation is not feature complete. Outer joins, distinct aggregates, date and decimal data types are currently not supported. However, all operations supported by the Table API are also supported by SQL.* +A `Table`, `DataSet`, `DataStream`, or external `TableSource` must be registered in the `TableEnvironment` in order to be accessible by a SQL query (see [Registering Tables](#registering-tables)). -In order to use a `Table`, `DataSet`, `DataStream`, or external `TableSource` in a SQL query, it has to be registered in the `TableEnvironment`, using a unique name. -A registered table can be accessed from a `TableEnvironment` using the `sql()` method of the `TableEnvironment`: - -- `tEnv.sql(SELECT * FROM tName)` executes the SQL query on the corresponding tables which were registered in a `TableEnvironment`. - -This method returns a new `Table` which can be converted back to a `DataSet`, or `DataStream`, or used in subsequent Table API queries. +*Note: Flink's SQL support is not feature complete, yet. Queries that include unsupported SQL features will cause a `TableException`. The limitations of SQL on batch and streaming tables are listed in the following sections.* +### SQL on Batch Tables
@@ -658,11 +654,12 @@ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // read a DataSet from an external source -DataSet> ds = env.readCsvFile(...); +DataSet> ds = env.readCsvFile(...); // register the DataSet as table "Orders" tableEnv.registerDataSet("Orders", ds, "user, product, amount"); -// run a SQL query and retrieve the result in a new Table -Table result = tableEnv.sql("SELECT SUM(amount) FROM Orders WHERE product = 10"); +// run a SQL query on the Table and retrieve the result as a new Table +Table result = tableEnv.sql( + "SELECT SUM(amount) FROM Orders WHERE product LIKE '%Rubber%'"); {% endhighlight %}
@@ -672,20 +669,75 @@ val env = ExecutionEnvironment.getExecutionEnvironment val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataSet from an external source -val ds = env.readCsvFile(...) +val ds: DataSet[(Long, String, Integer)] = env.readCsvFile(...) // register the DataSet under the name "Orders" tableEnv.registerDataSet("Orders", ds, 'user, 'product, 'amount) -// run a SQL query and retrieve the result in a new Table -val result = tableEnv.sql("SELECT SUM(amount) FROM Orders WHERE product = 10") +// run a SQL query on the Table and retrieve the result as a new Table +val result = tableEnv.sql( + "SELECT SUM(amount) FROM Orders WHERE product LIKE '%Rubber%'") {% endhighlight %}
-SQL queries can be executed on DataStream Tables by adding the `STREAM` SQL keyword before the table name. Please refer to the [Apache Calcite SQL Streaming documentation](https://calcite.apache.org/docs/stream.html) for more information on the Streaming SQL syntax. +#### Limitations + +The current version supports selection (filter), projection, inner equi-joins, grouping, non-distinct aggregates, and sorting on batch tables. + +Among others, the following SQL features are not supported, yet: + +- Time data types (`DATE`, `TIME`, `TIMESTAMP`, `INTERVAL`) and `DECIMAL` types +- Distinct aggregates (e.g., `COUNT(DISTINCT name)`) +- Outer joins +- Non-equi joins and Cartesian products +- Result selection by order position (`ORDER BY OFFSET FETCH`) +- Grouping sets +- Set operations except `UNION ALL` (`INTERSECT`, `UNION`, `EXCEPT`) + +*Note: Tables are joined in the order in which they are specified in the `FROM` clause. In some cases the table order must be manually tweaked to resolve Cartesian products. Certain rewrites during optimization (e.g., subquery decorrelation) can result in unsupported operations such as outer joins.* + +### SQL on Streaming Tables + +SQL queries can be executed on streaming Tables (Tables backed by `DataStream` or `StreamTableSource`) by using the `SELECT STREAM` keywords instead of `SELECT`. Please refer to the [Apache Calcite's Streaming SQL documentation](https://calcite.apache.org/docs/stream.html) for more information on the Streaming SQL syntax. + +
+
+{% highlight java %} +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); + +// ingest a DataStream from an external source +DataStream> ds = env.addSource(...); +// register the DataStream as table "Orders" +tableEnv.registerDataStream("Orders", ds, "user, product, amount"); +// run a SQL query on the Table and retrieve the result as a new Table +Table result = tableEnv.sql( + "SELECT STREAM product, amount FROM Orders WHERE product LIKE '%Rubber%'"); +{% endhighlight %} +
+ +
+{% highlight scala %} +val env = StreamExecutionEnvironment.getExecutionEnvironment +val tEnv = TableEnvironment.getTableEnvironment(env) + +// read a DataStream from an external source +val ds: DataStream[(Long, String, Integer)] = env.addSource(...) +// register the DataStream under the name "Orders" +tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount) +// run a SQL query on the Table and retrieve the result as a new Table +val result = tableEnv.sql( + "SELECT STREAM product, amount FROM Orders WHERE product LIKE '%Rubber%'") +{% endhighlight %} +
+
+ +#### Limitations + +The current version of streaming SQL only supports `SELECT`, `FROM`, `WHERE`, and `UNION` clauses. Aggregations or joins are not supported yet. {% top %} -Write Tables to external sinks +Writing Tables to External Sinks ---- A `Table` can be written to a `TableSink`, which is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ). A batch `Table` can only be written to a `BatchTableSink`, a streaming table requires a `StreamTableSink`. A `TableSink` can implement both interfaces at the same time. @@ -739,3 +791,4 @@ The Table API provides a configuration (the so-called `TableConfig`) to modify r ### Null Handling By default, the Table API supports `null` values. Null handling can be disabled to improve preformance by setting the `nullCheck` property in the `TableConfig` to `false`. +{% top %} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala index 4c8b37091278e..b1d55341533f5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala @@ -31,7 +31,6 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.api.table.explain.PlanJsonParser import org.apache.flink.api.table.expressions.Expression -import org.apache.flink.api.table.plan.PlanGenException import org.apache.flink.api.table.plan.logical.{CatalogNode, LogicalRelNode} import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetRel} import org.apache.flink.api.table.plan.rules.FlinkRuleSets @@ -252,15 +251,17 @@ abstract class BatchTableEnvironment( } catch { case e: CannotPlanException => - throw new PlanGenException( + throw new TableException( s"Cannot generate a valid execution plan for the given query: \n\n" + s"${RelOptUtil.toString(relNode)}\n" + - "Please consider filing a bug report.", e) + s"This exception indicates that the query uses an unsupported SQL feature.\n" + + s"Please check the documentation for the set of currently supported SQL features.") case t: TableException => - throw new PlanGenException( + throw new TableException( s"Cannot generate a valid execution plan for the given query: \n\n" + s"${RelOptUtil.toString(relNode)}\n" + - t.msg) + s"${t.msg}\n" + + s"Please check the documentation for the set of currently supported SQL features.") case a: AssertionError => throw a.getCause } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala index bacb587fa2052..daa74dadf2fc5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala @@ -27,7 +27,6 @@ import org.apache.calcite.tools.Programs import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.table.expressions.Expression -import org.apache.flink.api.table.plan.PlanGenException import org.apache.flink.api.table.plan.logical.{CatalogNode, LogicalRelNode} import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamConvention, DataStreamRel} import org.apache.flink.api.table.plan.rules.FlinkRuleSets @@ -255,10 +254,11 @@ abstract class StreamTableEnvironment( } catch { case e: CannotPlanException => - throw new PlanGenException( + throw new TableException( s"Cannot generate a valid execution plan for the given query: \n\n" + s"${RelOptUtil.toString(relNode)}\n" + - "Please consider filing a bug report.", e) + s"This exception indicates that the query uses an unsupported SQL feature.\n" + + s"Please check the documentation for the set of currently supported SQL features.") } dataStreamPlan match { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/PlanGenException.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/PlanGenException.scala deleted file mode 100644 index 2fd400da35843..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/PlanGenException.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.flink.api.table.plan - -class PlanGenException(message: String, exception: Exception) extends - RuntimeException(message: String, exception: Exception){ - - def this(message: String){ - this(message, null) - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala index 77e896fce2c93..bdc662a4d2ee8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala @@ -26,11 +26,10 @@ import org.apache.calcite.sql.`type`.SqlTypeName._ import org.apache.calcite.sql.`type`.{SqlTypeFactoryImpl, SqlTypeName} import org.apache.calcite.sql.fun._ import org.apache.flink.api.common.functions.{GroupReduceFunction, MapFunction} -import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.typeutils.{TypeConverter, RowTypeInfo} +import org.apache.flink.api.table.typeutils.TypeConverter import TypeConverter._ import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, TableConfig} +import org.apache.flink.api.table.{TableException, Row, TableConfig} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer @@ -96,7 +95,7 @@ object AggregateUtil { if (groupingOffsetMapping.length != groupings.length || aggOffsetMapping.length != namedAggregates.length) { - throw new PlanGenException("Could not find output field in input data type " + + throw new TableException("Could not find output field in input data type " + "or aggregate functions.") } @@ -138,11 +137,11 @@ object AggregateUtil { if (aggregateCall.getAggregation.isInstanceOf[SqlCountAggFunction]) { aggFieldIndexes(index) = 0 } else { - throw new PlanGenException("Aggregate fields should not be empty.") + throw new TableException("Aggregate fields should not be empty.") } } else { if (argList.size() > 1) { - throw new PlanGenException("Currently, do not support aggregate on multi fields.") + throw new TableException("Currently, do not support aggregate on multi fields.") } aggFieldIndexes(index) = argList.get(0) } @@ -163,7 +162,7 @@ object AggregateUtil { case DOUBLE => new DoubleSumAggregate case sqlType: SqlTypeName => - throw new PlanGenException("Sum aggregate does no support type:" + sqlType) + throw new TableException("Sum aggregate does no support type:" + sqlType) } } case _: SqlAvgAggFunction => { @@ -181,7 +180,7 @@ object AggregateUtil { case DOUBLE => new DoubleAvgAggregate case sqlType: SqlTypeName => - throw new PlanGenException("Avg aggregate does no support type:" + sqlType) + throw new TableException("Avg aggregate does no support type:" + sqlType) } } case sqlMinMaxFunction: SqlMinMaxAggFunction => { @@ -200,7 +199,7 @@ object AggregateUtil { case DOUBLE => new DoubleMinAggregate case sqlType: SqlTypeName => - throw new PlanGenException("Min aggregate does no support type:" + sqlType) + throw new TableException("Min aggregate does no support type:" + sqlType) } } else { sqlTypeName match { @@ -217,14 +216,14 @@ object AggregateUtil { case DOUBLE => new DoubleMaxAggregate case sqlType: SqlTypeName => - throw new PlanGenException("Max aggregate does no support type:" + sqlType) + throw new TableException("Max aggregate does no support type:" + sqlType) } } } case _: SqlCountAggFunction => aggregates(index) = new CountAggregate case unSupported: SqlAggFunction => - throw new PlanGenException("unsupported Function: " + unSupported.getName) + throw new TableException("unsupported Function: " + unSupported.getName) } setAggregateDataOffset(index) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala index 73c0724247471..01ec94aa93a91 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala @@ -23,8 +23,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{TableException, Row, TableEnvironment} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -213,7 +212,7 @@ class AggregationsITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testDistinctAggregate(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -228,7 +227,7 @@ class AggregationsITCase( tEnv.sql(sqlQuery).toDataSet[Row] } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testGroupedDistinctAggregate(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -243,7 +242,7 @@ class AggregationsITCase( tEnv.sql(sqlQuery).toDataSet[Row] } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testGroupingSetAggregate(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala index d388c33d9d8d5..50a4bdedb1a9a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala @@ -24,7 +24,6 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.plan.PlanGenException import org.apache.flink.api.table.{Row, TableEnvironment, TableException} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils @@ -190,7 +189,7 @@ class JoinITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testJoinNoEqualityPredicate(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -246,7 +245,7 @@ class JoinITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testFullOuterJoin(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -262,7 +261,7 @@ class JoinITCase( tEnv.sql(sqlQuery).toDataSet[Row].collect() } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testLeftOuterJoin(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment @@ -278,7 +277,7 @@ class JoinITCase( tEnv.sql(sqlQuery).toDataSet[Row].collect() } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testRightOuterJoin(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala index 7206be719be00..858f75a0dc40c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala @@ -25,8 +25,7 @@ import org.apache.flink.api.scala.batch.utils.SortTestUtils._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala._ -import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.{TableException, Row, TableEnvironment} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -61,7 +60,7 @@ class SortITCase( TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testOrderByOffset(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -73,7 +72,7 @@ class SortITCase( tEnv.sql(sqlQuery).toDataSet[Row] } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testOrderByFirst(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala index ae76acecbf773..126be4b61253f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala @@ -21,8 +21,7 @@ package org.apache.flink.api.scala.batch.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.plan.PlanGenException -import org.apache.flink.api.table.{ValidationException, Row, TableEnvironment} +import org.apache.flink.api.table.{TableException, ValidationException, Row, TableEnvironment} import org.apache.flink.api.table.expressions.Literal import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} @@ -140,7 +139,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .select('c, 'g) } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testNoEqualityJoinPredicate1(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) @@ -154,7 +153,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) .select('c, 'g).collect() } - @Test(expected = classOf[PlanGenException]) + @Test(expected = classOf[TableException]) def testNoEqualityJoinPredicate2(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) From 8572ecea85c1e07db3da20da45660db06d1cb085 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 24 May 2016 09:29:52 +0200 Subject: [PATCH 58/70] [FLINK-3960] ignore EventTimeWindowCheckpointingITCase for now Until FLINK-3960 is fixed, we need to disable this test to allow other tests to execute properly. This closes #2022 --- .../test/checkpointing/EventTimeWindowCheckpointingITCase.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index 199a6af985256..474fc607e2857 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -49,6 +49,7 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -71,6 +72,7 @@ */ @SuppressWarnings("serial") @RunWith(Parameterized.class) +@Ignore("Disabled because RocksDB fails with a segmentation fault. See FLINK-3960") public class EventTimeWindowCheckpointingITCase extends TestLogger { private static final int PARALLELISM = 4; From 3a6c0c8fd39e43303b0064d747d9265922ee3589 Mon Sep 17 00:00:00 2001 From: kl0u Date: Tue, 24 May 2016 13:56:05 +0200 Subject: [PATCH 59/70] [FLINK-3963] Removed shaded import This closes #2026 --- .../org/apache/flink/metrics/reporter/AbstractReporter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java index 1fe55df0b7377..8e5e2f02b441d 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java @@ -18,7 +18,7 @@ package org.apache.flink.metrics.reporter; -import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; +import org.jboss.netty.util.internal.ConcurrentHashMap; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; From ffb369e539094f994ad381de0494b6e533f58995 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 24 May 2016 15:59:19 +0200 Subject: [PATCH 60/70] [FLINK-3963] AbstractReporter uses wrong ConcurrentHashMap We should use java.util.concurrent.ConcurrentHashMap because Netty's ConcurrentHashMap is not available for Hadoop 1. Also, Netty's ConcurrentHashMap is merely a copy of Java's to support Java versions prior 1.5. --- .../org/apache/flink/metrics/reporter/AbstractReporter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java index 8e5e2f02b441d..271e91a6b17cd 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/reporter/AbstractReporter.java @@ -18,12 +18,12 @@ package org.apache.flink.metrics.reporter; -import org.jboss.netty.util.internal.ConcurrentHashMap; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; public abstract class AbstractReporter implements MetricReporter { From 1cc39f82599106f1f3f1b94c5e4a21126c16b55d Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Sun, 22 May 2016 16:46:43 +0200 Subject: [PATCH 61/70] [FLINK-3586] Fix potential overflow of Long AVG aggregation. - Add unit tests for Aggretates. This closes #2024 --- .../table/runtime/aggregate/Aggregate.scala | 61 ++++++---- .../runtime/aggregate/AggregateUtil.scala | 29 ++--- .../runtime/aggregate/AvgAggregate.scala | 51 ++++++-- .../runtime/aggregate/CountAggregate.scala | 6 +- .../runtime/aggregate/MaxAggregate.scala | 35 ++---- .../runtime/aggregate/MinAggregate.scala | 35 ++---- .../runtime/aggregate/SumAggregate.scala | 38 ++---- .../runtime/aggregate/AggregateTestBase.scala | 104 ++++++++++++++++ .../runtime/aggregate/AvgAggregateTest.scala | 115 ++++++++++++++++++ .../aggregate/CountAggregateTest.scala | 30 +++++ .../runtime/aggregate/MaxAggregateTest.scala | 93 ++++++++++++++ .../runtime/aggregate/MinAggregateTest.scala | 93 ++++++++++++++ .../runtime/aggregate/SumAggregateTest.scala | 89 ++++++++++++++ 13 files changed, 635 insertions(+), 144 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala index 496dcfb806654..1e91711c02eb6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.table.runtime.aggregate -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.table.Row /** @@ -43,47 +43,54 @@ import org.apache.flink.api.table.Row trait Aggregate[T] extends Serializable { /** - * Initiate the intermediate aggregate value in Row. - * @param intermediate - */ - def initiate(intermediate: Row): Unit + * Transform the aggregate field value into intermediate aggregate data. + * + * @param value The value to insert into the intermediate aggregate row. + * @param intermediate The intermediate aggregate row into which the value is inserted. + */ + def prepare(value: Any, intermediate: Row): Unit /** - * Transform the aggregate field value into intermediate aggregate data. - * @param value - * @param intermediate - */ - def prepare(value: Any, intermediate: Row): Unit + * Initiate the intermediate aggregate value in Row. + * + * @param intermediate The intermediate aggregate row to initiate. + */ + def initiate(intermediate: Row): Unit /** - * Merge intermediate aggregate data into aggregate buffer. - * @param intermediate - * @param buffer - */ + * Merge intermediate aggregate data into aggregate buffer. + * + * @param intermediate The intermediate aggregate row to merge. + * @param buffer The aggregate buffer into which the intermedidate is merged. + */ def merge(intermediate: Row, buffer: Row): Unit /** - * Calculate the final aggregated result based on aggregate buffer. - * @param buffer - * @return - */ + * Calculate the final aggregated result based on aggregate buffer. + * + * @param buffer The aggregate buffer from which the final aggregate is computed. + * @return The final result of the aggregate. + */ def evaluate(buffer: Row): T /** - * Intermediate aggregate value types. - * @return - */ - def intermediateDataType: Array[SqlTypeName] + * Intermediate aggregate value types. + * + * @return The types of the intermediate fields of this aggregate. + */ + def intermediateDataType: Array[TypeInformation[_]] /** - * Set the aggregate data offset in Row. - * @param aggOffset - */ + * Set the aggregate data offset in Row. + * + * @param aggOffset The offset of this aggregate in the intermediate aggregate rows. + */ def setAggOffsetInRow(aggOffset: Int) /** * Whether aggregate function support partial aggregate. - * @return - */ + * + * @return True if the aggregate supports partial aggregation, False otherwise. + */ def supportPartial: Boolean = false } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala index bdc662a4d2ee8..8222a2e2140e8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala @@ -26,8 +26,8 @@ import org.apache.calcite.sql.`type`.SqlTypeName._ import org.apache.calcite.sql.`type`.{SqlTypeFactoryImpl, SqlTypeName} import org.apache.calcite.sql.fun._ import org.apache.flink.api.common.functions.{GroupReduceFunction, MapFunction} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.table.typeutils.TypeConverter -import TypeConverter._ import org.apache.flink.api.table.typeutils.RowTypeInfo import org.apache.flink.api.table.{TableException, Row, TableConfig} @@ -73,15 +73,9 @@ object AggregateUtil { val aggFieldIndexes = aggregateFunctionsAndFieldIndexes._1 val aggregates = aggregateFunctionsAndFieldIndexes._2 - val bufferDataType: RelRecordType = + val mapReturnType: RowTypeInfo = createAggregateBufferDataType(groupings, aggregates, inputType) - val mapReturnType = determineReturnType( - bufferDataType, - Some(TypeConverter.DEFAULT_ROW_TYPE), - config.getNullCheck, - config.getEfficientTypeUsage) - val mapFunction = new AggregateMapFunction[Row, Row]( aggregates, aggFieldIndexes, groupings, mapReturnType.asInstanceOf[RowTypeInfo]).asInstanceOf[MapFunction[Any, Row]] @@ -240,25 +234,22 @@ object AggregateUtil { private def createAggregateBufferDataType( groupings: Array[Int], aggregates: Array[Aggregate[_]], - inputType: RelDataType): RelRecordType = { + inputType: RelDataType): RowTypeInfo = { // get the field data types of group keys. - val groupingTypes: Seq[RelDataTypeField] = groupings.map(inputType.getFieldList.get(_)) + val groupingTypes: Seq[TypeInformation[_]] = groupings + .map(inputType.getFieldList.get(_).getType.getSqlTypeName) + .map(TypeConverter.sqlTypeToTypeInfo) val aggPartialNameSuffix = "agg_buffer_" val factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT) - // get all the aggregate buffer value data type by their SqlTypeName. - val aggTypes: Seq[RelDataTypeField] = - aggregates.flatMap(_.intermediateDataType).zipWithIndex.map { - case (typeName: SqlTypeName, index: Int) => - val fieldDataType = factory.createSqlType(typeName) - new RelDataTypeFieldImpl(aggPartialNameSuffix + index, - groupings.length + index, fieldDataType) - } + // get all field data types of all intermediate aggregates + val aggTypes: Seq[TypeInformation[_]] = aggregates.flatMap(_.intermediateDataType) + // concat group key types and aggregation types val allFieldTypes = groupingTypes ++: aggTypes - val partialType = new RelRecordType(allFieldTypes.toList) + val partialType = new RowTypeInfo(allFieldTypes) partialType } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala index 8d3a45bd74f48..8cf181ac92077 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala @@ -18,8 +18,9 @@ package org.apache.flink.api.table.runtime.aggregate import com.google.common.math.LongMath -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.table.Row +import java.math.BigInteger abstract class AvgAggregate[T] extends Aggregate[T] { protected var partialSumIndex: Int = _ @@ -34,8 +35,6 @@ abstract class AvgAggregate[T] extends Aggregate[T] { } abstract class IntegralAvgAggregate[T] extends AvgAggregate[T] { - private final val intermediateType = Array(SqlTypeName.BIGINT, SqlTypeName.BIGINT) - override def initiate(partial: Row): Unit = { partial.setField(partialSumIndex, 0L) @@ -60,9 +59,9 @@ abstract class IntegralAvgAggregate[T] extends AvgAggregate[T] { buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount)) } - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array( + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO) def doPrepare(value: Any, partial: Row): Unit } @@ -113,21 +112,47 @@ class IntAvgAggregate extends IntegralAvgAggregate[Int] { class LongAvgAggregate extends IntegralAvgAggregate[Long] { + override def intermediateDataType = Array( + BasicTypeInfo.BIG_INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO) + + override def initiate(partial: Row): Unit = { + partial.setField(partialSumIndex, BigInteger.ZERO) + partial.setField(partialCountIndex, 0L) + } + + override def prepare(value: Any, partial: Row): Unit = { + if (value == null) { + partial.setField(partialSumIndex, BigInteger.ZERO) + partial.setField(partialCountIndex, 0L) + } else { + doPrepare(value, partial) + } + } + override def doPrepare(value: Any, partial: Row): Unit = { val input = value.asInstanceOf[Long] - partial.setField(partialSumIndex, input) + partial.setField(partialSumIndex, BigInteger.valueOf(input)) partial.setField(partialCountIndex, 1L) } + override def merge(partial: Row, buffer: Row): Unit = { + val partialSum = partial.productElement(partialSumIndex).asInstanceOf[BigInteger] + val partialCount = partial.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[BigInteger] + val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + buffer.setField(partialSumIndex, partialSum.add(bufferSum)) + buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount)) + } + override def evaluate(buffer: Row): Long = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Long] + val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[BigInteger] val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] - (bufferSum / bufferCount) + bufferSum.divide(BigInteger.valueOf(bufferCount)).longValue() } } abstract class FloatingAvgAggregate[T: Numeric] extends AvgAggregate[T] { - private val partialType = Array(SqlTypeName.DOUBLE, SqlTypeName.BIGINT) override def initiate(partial: Row): Unit = { partial.setField(partialSumIndex, 0D) @@ -153,9 +178,9 @@ abstract class FloatingAvgAggregate[T: Numeric] extends AvgAggregate[T] { buffer.setField(partialCountIndex, partialCount + bufferCount) } - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array( + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO) def doPrepare(value: Any, partial: Row): Unit } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala index d615088a546f0..d9f288a2f7832 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.table.runtime.aggregate -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.table.Row class CountAggregate extends Aggregate[Long] { @@ -45,9 +45,7 @@ class CountAggregate extends Aggregate[Long] { } } - override def intermediateDataType: Array[SqlTypeName] = { - Array(SqlTypeName.BIGINT) - } + override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO) override def supportPartial: Boolean = true diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala index fde1b53aa016f..8f491f26c8d24 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.table.runtime.aggregate -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.table.Row abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { @@ -27,6 +27,7 @@ abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { /** * Accessed in MapFunction, prepare the input of partial aggregate. + * * @param value * @param intermediate */ @@ -41,6 +42,7 @@ abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { /** * Accessed in CombineFunction and GroupReduceFunction, merge partial * aggregate result into aggregate buffer. + * * @param intermediate * @param buffer */ @@ -52,6 +54,7 @@ abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { /** * Return the final aggregated result based on aggregate buffer. + * * @param buffer * @return */ @@ -67,11 +70,8 @@ abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { } class ByteMaxAggregate extends MaxAggregate[Byte] { - private val intermediateType = Array(SqlTypeName.TINYINT) - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(maxIndex, Byte.MinValue) @@ -79,11 +79,8 @@ class ByteMaxAggregate extends MaxAggregate[Byte] { } class ShortMaxAggregate extends MaxAggregate[Short] { - private val intermediateType = Array(SqlTypeName.SMALLINT) - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(maxIndex, Short.MinValue) @@ -91,11 +88,8 @@ class ShortMaxAggregate extends MaxAggregate[Short] { } class IntMaxAggregate extends MaxAggregate[Int] { - private val intermediateType = Array(SqlTypeName.INTEGER) - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(maxIndex, Int.MinValue) @@ -103,11 +97,8 @@ class IntMaxAggregate extends MaxAggregate[Int] { } class LongMaxAggregate extends MaxAggregate[Long] { - private val intermediateType = Array(SqlTypeName.BIGINT) - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(maxIndex, Long.MinValue) @@ -115,11 +106,8 @@ class LongMaxAggregate extends MaxAggregate[Long] { } class FloatMaxAggregate extends MaxAggregate[Float] { - private val intermediateType = Array(SqlTypeName.FLOAT) - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(maxIndex, Float.MinValue) @@ -127,11 +115,8 @@ class FloatMaxAggregate extends MaxAggregate[Float] { } class DoubleMaxAggregate extends MaxAggregate[Double] { - private val intermediateType = Array(SqlTypeName.DOUBLE) - override def intermediateDataType: Array[SqlTypeName] = { - intermediateType - } + override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(maxIndex, Double.MinValue) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala index 7cc1b4829c811..e78fb00cccd35 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.table.runtime.aggregate -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.table.Row abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ @@ -27,6 +27,7 @@ abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ /** * Accessed in MapFunction, prepare the input of partial aggregate. + * * @param value * @param partial */ @@ -41,6 +42,7 @@ abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ /** * Accessed in CombineFunction and GroupReduceFunction, merge partial * aggregate result into aggregate buffer. + * * @param partial * @param buffer */ @@ -52,6 +54,7 @@ abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ /** * Return the final aggregated result based on aggregate buffer. + * * @param buffer * @return */ @@ -67,11 +70,8 @@ abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ } class ByteMinAggregate extends MinAggregate[Byte] { - private val partialType = Array(SqlTypeName.TINYINT) - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(minIndex, Byte.MaxValue) @@ -79,11 +79,8 @@ class ByteMinAggregate extends MinAggregate[Byte] { } class ShortMinAggregate extends MinAggregate[Short] { - private val partialType = Array(SqlTypeName.SMALLINT) - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(minIndex, Short.MaxValue) @@ -91,11 +88,8 @@ class ShortMinAggregate extends MinAggregate[Short] { } class IntMinAggregate extends MinAggregate[Int] { - private val partialType = Array(SqlTypeName.INTEGER) - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(minIndex, Int.MaxValue) @@ -103,11 +97,8 @@ class IntMinAggregate extends MinAggregate[Int] { } class LongMinAggregate extends MinAggregate[Long] { - private val partialType = Array(SqlTypeName.BIGINT) - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(minIndex, Long.MaxValue) @@ -115,11 +106,8 @@ class LongMinAggregate extends MinAggregate[Long] { } class FloatMinAggregate extends MinAggregate[Float] { - private val partialType = Array(SqlTypeName.FLOAT) - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(minIndex, Float.MaxValue) @@ -127,11 +115,8 @@ class FloatMinAggregate extends MinAggregate[Float] { } class DoubleMinAggregate extends MinAggregate[Double] { - private val partialType = Array(SqlTypeName.DOUBLE) - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO) override def initiate(intermediate: Row): Unit = { intermediate.setField(minIndex, Double.MaxValue) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala index 25ef3443604ee..b4c56fe4030d0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.table.runtime.aggregate -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.table.Row abstract class SumAggregate[T: Numeric] @@ -57,49 +57,25 @@ abstract class SumAggregate[T: Numeric] } class ByteSumAggregate extends SumAggregate[Byte] { - private val partialType = Array(SqlTypeName.TINYINT) - - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO) } class ShortSumAggregate extends SumAggregate[Short] { - private val partialType = Array(SqlTypeName.SMALLINT) - - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO) } class IntSumAggregate extends SumAggregate[Int] { - private val partialType = Array(SqlTypeName.INTEGER) - - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO) } class LongSumAggregate extends SumAggregate[Long] { - private val partialType = Array(SqlTypeName.BIGINT) - - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO) } class FloatSumAggregate extends SumAggregate[Float] { - private val partialType = Array(SqlTypeName.FLOAT) - - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO) } class DoubleSumAggregate extends SumAggregate[Double] { - private val partialType = Array(SqlTypeName.DOUBLE) - - override def intermediateDataType: Array[SqlTypeName] = { - partialType - } + override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala new file mode 100644 index 0000000000000..78d5f8c0bac42 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.runtime.aggregate + +import org.apache.flink.api.table.Row +import org.junit.Test +import org.junit.Assert.assertEquals + +abstract class AggregateTestBase[T] { + + private val offset = 2 + private val rowArity: Int = offset + aggregator.intermediateDataType.length + + def inputValueSets: Seq[Seq[_]] + + def expectedResults: Seq[T] + + def aggregator: Aggregate[T] + + private def createAggregator(): Aggregate[T] = { + val agg = aggregator + agg.setAggOffsetInRow(offset) + agg + } + + private def createRow(): Row = { + new Row(rowArity) + } + + @Test + def testAggregate(): Unit = { + + // iterate over input sets + for((vals, expected) <- inputValueSets.zip(expectedResults)) { + + // prepare mapper + val rows: Seq[Row] = prepare(vals) + + val result = if (aggregator.supportPartial) { + // test with combiner + val (firstVals, secondVals) = rows.splitAt(rows.length / 2) + val combined = partialAgg(firstVals) :: partialAgg(secondVals) :: Nil + finalAgg(combined) + + } else { + // test without combiner + finalAgg(rows) + } + + assertEquals(expected, result) + + } + } + + private def prepare(vals: Seq[_]): Seq[Row] = { + + val agg = createAggregator() + + vals.map { v => + val row = createRow() + agg.prepare(v, row) + row + } + } + + private def partialAgg(rows: Seq[Row]): Row = { + + val agg = createAggregator() + val aggBuf = createRow() + + agg.initiate(aggBuf) + rows.foreach(v => agg.merge(v, aggBuf)) + + aggBuf + } + + private def finalAgg(rows: Seq[Row]): T = { + + val agg = createAggregator() + val aggBuf = createRow() + + agg.initiate(aggBuf) + rows.foreach(v => agg.merge(v, aggBuf)) + + agg.evaluate(partialAgg(rows)) + } + +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala new file mode 100644 index 0000000000000..2575fa2d4e369 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.runtime.aggregate + +abstract class AvgAggregateTestBase[T: Numeric] extends AggregateTestBase[T] { + + private val numeric: Numeric[T] = implicitly[Numeric[T]] + + def minVal: T + def maxVal: T + + override def inputValueSets: Seq[Seq[T]] = Seq( + Seq( + minVal, + minVal, + null.asInstanceOf[T], + minVal, + minVal, + null.asInstanceOf[T], + minVal, + minVal, + minVal + ), + Seq( + maxVal, + maxVal, + null.asInstanceOf[T], + maxVal, + maxVal, + null.asInstanceOf[T], + maxVal, + maxVal, + maxVal + ), + Seq( + minVal, + maxVal, + null.asInstanceOf[T], + numeric.fromInt(0), + numeric.negate(maxVal), + numeric.negate(minVal), + null.asInstanceOf[T] + ) + ) + + override def expectedResults: Seq[T] = Seq( + minVal, + maxVal, + numeric.fromInt(0) + ) +} + +class ByteAvgAggregateTest extends AvgAggregateTestBase[Byte] { + + override def minVal = (Byte.MinValue + 1).toByte + override def maxVal = (Byte.MaxValue - 1).toByte + + override def aggregator = new ByteAvgAggregate() +} + +class ShortAvgAggregateTest extends AvgAggregateTestBase[Short] { + + override def minVal = (Short.MinValue + 1).toShort + override def maxVal = (Short.MaxValue - 1).toShort + + override def aggregator = new ShortAvgAggregate() +} + +class IntAvgAggregateTest extends AvgAggregateTestBase[Int] { + + override def minVal = Int.MinValue + 1 + override def maxVal = Int.MaxValue - 1 + + override def aggregator = new IntAvgAggregate() +} + +class LongAvgAggregateTest extends AvgAggregateTestBase[Long] { + + override def minVal = Long.MinValue + 1 + override def maxVal = Long.MaxValue - 1 + + override def aggregator = new LongAvgAggregate() +} + +class FloatAvgAggregateTest extends AvgAggregateTestBase[Float] { + + override def minVal = Float.MinValue + override def maxVal = Float.MaxValue + + override def aggregator = new FloatAvgAggregate() +} + +class DoubleAvgAggregateTest extends AvgAggregateTestBase[Double] { + + override def minVal = Float.MinValue + override def maxVal = Float.MaxValue + + override def aggregator = new DoubleAvgAggregate() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala new file mode 100644 index 0000000000000..ce27d7cd3afdd --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.runtime.aggregate + +class CountAggregateTest extends AggregateTestBase[Long] { + + override def inputValueSets: Seq[Seq[_]] = Seq( + Seq("a", "b", null, "c", null, "d", "e", null, "f") + ) + + override def expectedResults: Seq[Long] = Seq(6L) + + override def aggregator: Aggregate[Long] = new CountAggregate() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala new file mode 100644 index 0000000000000..f3951e4dc3618 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.runtime.aggregate + +abstract class MaxAggregateTestBase[T: Numeric] extends AggregateTestBase[T] { + + private val numeric: Numeric[T] = implicitly[Numeric[T]] + + def minVal: T + def maxVal: T + + override def inputValueSets: Seq[Seq[T]] = Seq( + Seq( + numeric.fromInt(1), + null.asInstanceOf[T], + maxVal, + numeric.fromInt(-99), + numeric.fromInt(3), + numeric.fromInt(56), + numeric.fromInt(0), + minVal, + numeric.fromInt(-20), + numeric.fromInt(17), + null.asInstanceOf[T] + ) + ) + + override def expectedResults: Seq[T] = Seq(maxVal) +} + +class ByteMaxAggregateTest extends MaxAggregateTestBase[Byte] { + + override def minVal = (Byte.MinValue + 1).toByte + override def maxVal = (Byte.MaxValue - 1).toByte + + override def aggregator: Aggregate[Byte] = new ByteMaxAggregate() +} + +class ShortMaxAggregateTest extends MaxAggregateTestBase[Short] { + + override def minVal = (Short.MinValue + 1).toShort + override def maxVal = (Short.MaxValue - 1).toShort + + override def aggregator: Aggregate[Short] = new ShortMaxAggregate() +} + +class IntMaxAggregateTest extends MaxAggregateTestBase[Int] { + + override def minVal = Int.MinValue + 1 + override def maxVal = Int.MaxValue - 1 + + override def aggregator: Aggregate[Int] = new IntMaxAggregate() +} + +class LongMaxAggregateTest extends MaxAggregateTestBase[Long] { + + override def minVal = Long.MinValue + 1 + override def maxVal = Long.MaxValue - 1 + + override def aggregator: Aggregate[Long] = new LongMaxAggregate() +} + +class FloatMaxAggregateTest extends MaxAggregateTestBase[Float] { + + override def minVal = Float.MinValue / 2 + override def maxVal = Float.MaxValue / 2 + + override def aggregator: Aggregate[Float] = new FloatMaxAggregate() +} + +class DoubleMaxAggregateTest extends MaxAggregateTestBase[Double] { + + override def minVal = Double.MinValue / 2 + override def maxVal = Double.MaxValue / 2 + + override def aggregator: Aggregate[Double] = new DoubleMaxAggregate() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala new file mode 100644 index 0000000000000..3a4b111d53fff --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.runtime.aggregate + +abstract class MinAggregateTestBase[T: Numeric] extends AggregateTestBase[T] { + + private val numeric: Numeric[T] = implicitly[Numeric[T]] + + def minVal: T + def maxVal: T + + override def inputValueSets: Seq[Seq[T]] = Seq( + Seq( + numeric.fromInt(1), + null.asInstanceOf[T], + maxVal, + numeric.fromInt(-99), + numeric.fromInt(3), + numeric.fromInt(56), + numeric.fromInt(0), + minVal, + numeric.fromInt(-20), + numeric.fromInt(17), + null.asInstanceOf[T] + ) + ) + + override def expectedResults: Seq[T] = Seq(minVal) +} + +class ByteMinAggregateTest extends MinAggregateTestBase[Byte] { + + override def minVal = (Byte.MinValue + 1).toByte + override def maxVal = (Byte.MaxValue - 1).toByte + + override def aggregator: Aggregate[Byte] = new ByteMinAggregate() +} + +class ShortMinAggregateTest extends MinAggregateTestBase[Short] { + + override def minVal = (Short.MinValue + 1).toShort + override def maxVal = (Short.MaxValue - 1).toShort + + override def aggregator: Aggregate[Short] = new ShortMinAggregate() +} + +class IntMinAggregateTest extends MinAggregateTestBase[Int] { + + override def minVal = Int.MinValue + 1 + override def maxVal = Int.MaxValue - 1 + + override def aggregator: Aggregate[Int] = new IntMinAggregate() +} + +class LongMinAggregateTest extends MinAggregateTestBase[Long] { + + override def minVal = Long.MinValue + 1 + override def maxVal = Long.MaxValue - 1 + + override def aggregator: Aggregate[Long] = new LongMinAggregate() +} + +class FloatMinAggregateTest extends MinAggregateTestBase[Float] { + + override def minVal = Float.MinValue / 2 + override def maxVal = Float.MaxValue / 2 + + override def aggregator: Aggregate[Float] = new FloatMinAggregate() +} + +class DoubleMinAggregateTest extends MinAggregateTestBase[Double] { + + override def minVal = Double.MinValue / 2 + override def maxVal = Double.MaxValue / 2 + + override def aggregator: Aggregate[Double] = new DoubleMinAggregate() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala new file mode 100644 index 0000000000000..f5de3fcee7f40 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.runtime.aggregate + +abstract class SumAggregateTestBase[T: Numeric] extends AggregateTestBase[T] { + + private val numeric: Numeric[T] = implicitly[Numeric[T]] + + def maxVal: T + private val minVal = numeric.negate(maxVal) + + override def inputValueSets: Seq[Seq[T]] = Seq( + Seq( + minVal, + numeric.fromInt(1), + null.asInstanceOf[T], + numeric.fromInt(2), + numeric.fromInt(3), + numeric.fromInt(4), + numeric.fromInt(5), + numeric.fromInt(-10), + numeric.fromInt(-20), + numeric.fromInt(17), + null.asInstanceOf[T], + maxVal + ) + ) + + override def expectedResults: Seq[T] = Seq(numeric.fromInt(2)) + +} + +class ByteSumAggregateTest extends SumAggregateTestBase[Byte] { + + override def maxVal = (Byte.MaxValue / 2).toByte + + override def aggregator: Aggregate[Byte] = new ByteSumAggregate +} + +class ShortSumAggregateTest extends SumAggregateTestBase[Short] { + + override def maxVal = (Short.MaxValue / 2).toShort + + override def aggregator: Aggregate[Short] = new ShortSumAggregate +} + +class IntSumAggregateTest extends SumAggregateTestBase[Int] { + + override def maxVal = Int.MaxValue / 2 + + override def aggregator: Aggregate[Int] = new IntSumAggregate +} + +class LongSumAggregateTest extends SumAggregateTestBase[Long] { + + override def maxVal = Long.MaxValue / 2 + + override def aggregator: Aggregate[Long] = new LongSumAggregate +} + +class FloatSumAggregateTest extends SumAggregateTestBase[Float] { + + override def maxVal = 12345.6789f + + override def aggregator: Aggregate[Float] = new FloatSumAggregate +} + +class DoubleSumAggregateTest extends SumAggregateTestBase[Double] { + + override def maxVal = 12345.6789d + + override def aggregator: Aggregate[Double] = new DoubleSumAggregate +} From 749f0cf7d748d29a3df92ebfe387273eedc69766 Mon Sep 17 00:00:00 2001 From: gallenvara Date: Mon, 2 May 2016 17:58:17 +0800 Subject: [PATCH 62/70] [FLINK-2044] [gelly] Implementation of HITS algorithm. This closes #1956 --- docs/apis/batch/libs/gelly.md | 18 + .../flink/graph/examples/data/HITSData.java | 71 ++++ .../graph/library/HITSAlgorithmITCase.java | 133 ++++++++ .../flink/graph/library/HITSAlgorithm.java | 312 ++++++++++++++++++ 4 files changed, 534 insertions(+) create mode 100644 flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/HITSData.java create mode 100644 flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/HITSAlgorithmITCase.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/HITSAlgorithm.java diff --git a/docs/apis/batch/libs/gelly.md b/docs/apis/batch/libs/gelly.md index 0790c2974d09e..0d3e59409860e 100644 --- a/docs/apis/batch/libs/gelly.md +++ b/docs/apis/batch/libs/gelly.md @@ -1830,6 +1830,7 @@ Gelly has a growing collection of graph algorithms for easily analyzing large-sc * [GSA Single Source Shortest Paths](#gsa-single-source-shortest-paths) * [GSA Triangle Count](#gsa-triangle-count) * [Triangle Enumerator](#triangle-enumerator) +* [Hyperlink-Induced Topic Search](#hyperlink-induced-topic-search) * [Summarization](#summarization) * [Jaccard Index](#jaccard-index) * [Local Clustering Coefficient](#local-clustering-coefficient) @@ -2028,6 +2029,23 @@ This implementation extends the basic algorithm by computing output degrees of e The algorithm takes a directed graph as input and outputs a `DataSet` of `Tuple3`. The Vertex ID type has to be `Comparable`. Each `Tuple3` corresponds to a triangle, with the fields containing the IDs of the vertices forming the triangle. +### Hyperlink-Induced Topic Search + +#### Overview +[Hyperlink-Induced Topic Search](http://www.cs.cornell.edu/home/kleinber/auth.pdf) (HITS, or "Hubs and Authorities") +computes two interdependent scores for every vertex in a directed graph. Good hubs are those which point to many +good authorities and good authorities are those pointed to by many good hubs. + +#### Details +HITS ranking relies on an iterative method converging to a stationary solution. Each vertex in the directed graph is assigned same non-negative +hub and authority scores. Then the algorithm iteratively updates the scores until termination. Current implementation divides the iteration +into two phases, authority scores can be computed until hub scores updating and normalising finished, hub scores can be computed until +authority scores updating and normalising finished. + +#### Usage +The algorithm takes a directed graph as input and outputs a `DataSet` of vertices, where the vertex value is a `Tuple2` +containing the hub and authority score after maximum iterations. + ### Summarization #### Overview diff --git a/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/HITSData.java b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/HITSData.java new file mode 100644 index 0000000000000..7f408560df628 --- /dev/null +++ b/flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/data/HITSData.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.examples.data; + +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.Vertex; +import org.apache.flink.types.NullValue; + +import java.util.ArrayList; +import java.util.List; + +/** + * Provides the data set used for the HITS test program. + */ +public class HITSData { + + public static final String VALUE_AFTER_10_ITERATIONS = "1,0.70710678,3.12608866E-8\n" + + "2,1.29486832E-8,0.70710678\n" + + "3,1.29486832E-8,0.49999999\n" + + "4,0.50000001,0.49999999\n" + + "5,0.50000001,3.12608866E-8\n"; + + + private HITSData() {} + + public static final DataSet> getVertexDataSet(ExecutionEnvironment env) { + + List> vertices = new ArrayList<>(); + vertices.add(new Vertex<>(1L, 1.0)); + vertices.add(new Vertex<>(2L, 2.0)); + vertices.add(new Vertex<>(3L, 3.0)); + vertices.add(new Vertex<>(4L, 4.0)); + vertices.add(new Vertex<>(5L, 5.0)); + + return env.fromCollection(vertices); + } + + public static final DataSet> getEdgeDataSet(ExecutionEnvironment env) { + + List> edges = new ArrayList<>(); + edges.add(new Edge<>(2L, 1L, NullValue.getInstance())); + edges.add(new Edge<>(5L, 2L, NullValue.getInstance())); + edges.add(new Edge<>(5L, 4L, NullValue.getInstance())); + edges.add(new Edge<>(4L, 3L, NullValue.getInstance())); + edges.add(new Edge<>(4L, 2L, NullValue.getInstance())); + edges.add(new Edge<>(1L, 4L, NullValue.getInstance())); + edges.add(new Edge<>(1L, 2L, NullValue.getInstance())); + edges.add(new Edge<>(1L, 3L, NullValue.getInstance())); + edges.add(new Edge<>(3L, 5L, NullValue.getInstance())); + + return env.fromCollection(edges); + } +} diff --git a/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/HITSAlgorithmITCase.java b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/HITSAlgorithmITCase.java new file mode 100644 index 0000000000000..019b8513cfb4b --- /dev/null +++ b/flink-libraries/flink-gelly-examples/src/test/java/org/apache/flink/graph/library/HITSAlgorithmITCase.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.library; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.examples.data.HITSData; +import org.apache.flink.test.util.MultipleProgramsTestBase; +import org.apache.flink.types.DoubleValue; +import org.apache.flink.types.NullValue; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.List; + +@RunWith(Parameterized.class) +public class HITSAlgorithmITCase extends MultipleProgramsTestBase{ + + public HITSAlgorithmITCase(TestExecutionMode mode) { + super(mode); + } + + @Test + public void testHITSWithTenIterations() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.fromDataSet( + HITSData.getVertexDataSet(env), + HITSData.getEdgeDataSet(env), + env); + + List>> result = graph.run(new HITSAlgorithm(10)).collect(); + + compareWithDelta(result, 1e-7); + } + + @Test + public void testHITSWithTenIterationsAndNumOfVertices() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.fromDataSet( + HITSData.getVertexDataSet(env), + HITSData.getEdgeDataSet(env), + env); + + List>> result = graph.run(new HITSAlgorithm(10, 5)).collect(); + + compareWithDelta(result, 1e-7); + } + + @Test + public void testHITSWithConvergeThreshold() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.fromDataSet( + HITSData.getVertexDataSet(env), + HITSData.getEdgeDataSet(env), + env); + + List>> result = graph.run(new HITSAlgorithm(1e-7)).collect(); + + compareWithDelta(result, 1e-7); + } + + @Test + public void testHITSWithConvergeThresholdAndNumOfVertices() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + Graph graph = Graph.fromDataSet( + HITSData.getVertexDataSet(env), + HITSData.getEdgeDataSet(env), + env); + + List>> result = graph.run(new HITSAlgorithm(1e-7, 5)).collect(); + + compareWithDelta(result, 1e-7); + } + + private void compareWithDelta(List>> result, double delta) { + + String resultString = ""; + for (Vertex> v : result) { + resultString += v.f0.toString() + "," + v.f1.f0.toString() + "," + v.f1.f1.toString() +"\n"; + } + + String expectedResult = HITSData.VALUE_AFTER_10_ITERATIONS; + + String[] expected = expectedResult.isEmpty() ? new String[0] : expectedResult.split("\n"); + + String[] resultArray = resultString.isEmpty() ? new String[0] : resultString.split("\n"); + + Arrays.sort(expected); + Arrays.sort(resultArray); + + for (int i = 0; i < expected.length; i++) { + String[] expectedFields = expected[i].split(","); + String[] resultFields = resultArray[i].split(","); + + double expectedHub = Double.parseDouble(expectedFields[1]); + double resultHub = Double.parseDouble(resultFields[1]); + + double expectedAuthority = Double.parseDouble(expectedFields[2]); + double resultAuthority = Double.parseDouble(resultFields[2]); + + Assert.assertTrue("Values differ by more than the permissible delta", + Math.abs(expectedHub - resultHub) < delta); + + Assert.assertTrue("Values differ by more than the permissible delta", + Math.abs(expectedAuthority - resultAuthority) < delta); + } + } +} diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/HITSAlgorithm.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/HITSAlgorithm.java new file mode 100644 index 0000000000000..1ea367eca7627 --- /dev/null +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/HITSAlgorithm.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.graph.library; + +import org.apache.flink.api.common.aggregators.DoubleSumAggregator; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.DataSet; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.graph.Edge; +import org.apache.flink.graph.EdgeDirection; +import org.apache.flink.graph.Graph; +import org.apache.flink.graph.GraphAlgorithm; +import org.apache.flink.graph.Vertex; +import org.apache.flink.graph.spargel.MessageIterator; +import org.apache.flink.graph.spargel.MessagingFunction; +import org.apache.flink.graph.spargel.ScatterGatherConfiguration; +import org.apache.flink.graph.spargel.VertexUpdateFunction; +import org.apache.flink.types.DoubleValue; +import org.apache.flink.util.Preconditions; + +/** + * This is an implementation of HITS algorithm, using a scatter-gather iteration. + * The user can define the maximum number of iterations. HITS algorithm is determined by two parameters, + * hubs and authorities. A good hub represents a page that points to many other pages, and a good authority + * represented a page that is linked by many different hubs. + * Each vertex has a value of Tuple2 type, the first field is hub score and the second field is authority score. + * The implementation sets same score to every vertex and adds the reverse edge to every edge at the beginning. + * If the number of vertices of the input graph is known, it should be provided as a parameter + * to speed up computation. Otherwise, the algorithm will first execute a job to count the vertices. + *

+ * + * @see HITS Algorithm + */ +public class HITSAlgorithm implements GraphAlgorithm>>> { + + private final static int MAXIMUMITERATION = (Integer.MAX_VALUE - 1) / 2; + private final static double MINIMUMTHRESHOLD = 1e-9; + + private int maxIterations; + private long numberOfVertices; + private double convergeThreshold; + + /** + * Create an instance of HITS algorithm. + * + * @param maxIterations the maximum number of iterations + */ + public HITSAlgorithm(int maxIterations) { + this(maxIterations, MINIMUMTHRESHOLD); + } + + /** + * Create an instance of HITS algorithm. + * + * @param convergeThreshold convergence threshold for sum of scores to control whether the iteration should be stopped + */ + public HITSAlgorithm(double convergeThreshold) { + this(MAXIMUMITERATION, convergeThreshold); + } + + /** + * Create an instance of HITS algorithm. + * + * @param maxIterations the maximum number of iterations + * @param numberOfVertices the number of vertices in the graph + */ + public HITSAlgorithm(int maxIterations, long numberOfVertices) { + this(maxIterations, MINIMUMTHRESHOLD, numberOfVertices); + } + + /** + * Create an instance of HITS algorithm. + * + * @param convergeThreshold convergence threshold for sum of scores to control whether the iteration should be stopped + * @param numberOfVertices the number of vertices in the graph + */ + public HITSAlgorithm(double convergeThreshold, long numberOfVertices) { + this(MAXIMUMITERATION, convergeThreshold, numberOfVertices); + } + + /** + * Creates an instance of HITS algorithm. + * + * @param maxIterations the maximum number of iterations + * @param convergeThreshold convergence threshold for sum of scores to control whether the iteration should be stopped + */ + public HITSAlgorithm(int maxIterations, double convergeThreshold) { + Preconditions.checkArgument(maxIterations > 0, "Number of iterations must be greater than zero."); + Preconditions.checkArgument(convergeThreshold > 0.0, "Convergence threshold must be greater than zero."); + this.maxIterations = maxIterations * 2 + 1; + this.convergeThreshold = convergeThreshold; + } + + /** + * Creates an instance of HITS algorithm. + * + * @param maxIterations the maximum number of iterations + * @param convergeThreshold convergence threshold for sum of scores to control whether the iteration should be stopped + * @param numberOfVertices the number of vertices in the graph + */ + public HITSAlgorithm(int maxIterations, double convergeThreshold, long numberOfVertices) { + this(maxIterations, convergeThreshold); + Preconditions.checkArgument(numberOfVertices > 0, "Number of vertices must be greater than zero."); + this.numberOfVertices = numberOfVertices; + } + + @Override + public DataSet>> run(Graph graph) throws Exception { + + if (numberOfVertices == 0) { + numberOfVertices = graph.numberOfVertices(); + } + + Graph, Boolean> newGraph = graph + .mapEdges(new AuthorityEdgeMapper()) + .union(graph.reverse().mapEdges(new HubEdgeMapper())) + .mapVertices(new VertexInitMapper()); + + ScatterGatherConfiguration parameter = new ScatterGatherConfiguration(); + parameter.setDirection(EdgeDirection.OUT); + parameter.registerAggregator("updatedValueSum", new DoubleSumAggregator()); + parameter.registerAggregator("authorityValueSum", new DoubleSumAggregator()); + parameter.registerAggregator("diffValueSum", new DoubleSumAggregator()); + + return newGraph + .runScatterGatherIteration(new VertexUpdate(maxIterations, convergeThreshold, numberOfVertices), + new MessageUpdate(maxIterations), maxIterations, parameter) + .getVertices(); + } + + /** + * Function that updates the value of a vertex by summing up the partial + * values from all messages and normalize the value. + */ + @SuppressWarnings("serial") + public static final class VertexUpdate extends VertexUpdateFunction, Double> { + private int maxIteration; + private double convergeThreshold; + private long numberOfVertices; + private DoubleSumAggregator updatedValueSumAggregator; + private DoubleSumAggregator authoritySumAggregator; + private DoubleSumAggregator diffSumAggregator; + + public VertexUpdate(int maxIteration, double convergeThreshold, long numberOfVertices) { + this.maxIteration = maxIteration; + this.convergeThreshold = convergeThreshold; + this.numberOfVertices = numberOfVertices; + } + + @Override + public void preSuperstep() { + updatedValueSumAggregator = getIterationAggregator("updatedValueSum"); + authoritySumAggregator = getIterationAggregator("authorityValueSum"); + diffSumAggregator = getIterationAggregator("diffValueSum"); + } + + @Override + public void updateVertex(Vertex> vertex, MessageIterator inMessages) { + double updateValue = 0; + + for (double element : inMessages) { + if (getSuperstepNumber() == maxIteration) { + updateValue = element; + break; + } + updateValue += element; + } + updatedValueSumAggregator.aggregate(Math.pow(updateValue, 2)); + + // in the first iteration, no aggregation to call, init sum with value of vertex + double iterationValueSum = 1.0; + + DoubleValue newHubValue = vertex.getValue().f0; + DoubleValue newAuthorityValue = vertex.getValue().f1; + + if (getSuperstepNumber() > 1) { + iterationValueSum = Math.sqrt(((DoubleValue) getPreviousIterationAggregate("updatedValueSum")).getValue()); + } + if (getSuperstepNumber() < maxIteration) { + if (getSuperstepNumber() % 2 == 1) { + + //in the first iteration, the diff is the authority value of each vertex + double previousAuthAverage = 1.0; + double diffValueSum = 1.0 * numberOfVertices; + if (getSuperstepNumber() > 1) { + previousAuthAverage = ((DoubleValue) getPreviousIterationAggregate("authorityValueSum")).getValue() / numberOfVertices; + diffValueSum = ((DoubleValue) getPreviousIterationAggregate("diffValueSum")).getValue(); + } + authoritySumAggregator.aggregate(previousAuthAverage); + + if (diffValueSum > convergeThreshold) { + newHubValue.setValue(newHubValue.getValue() / iterationValueSum); + newAuthorityValue.setValue(updateValue); + } else { + + //scores are converged and stop iteration + maxIteration = getSuperstepNumber(); + newHubValue.setValue(newHubValue.getValue() / iterationValueSum); + } + } else { + newHubValue.setValue(updateValue); + newAuthorityValue.setValue(newAuthorityValue.getValue() / iterationValueSum); + authoritySumAggregator.aggregate(newAuthorityValue.getValue()); + double previousAuthAverage = ((DoubleValue) getPreviousIterationAggregate("authorityValueSum")).getValue() / numberOfVertices; + + // count the diff value of sum of authority scores + diffSumAggregator.aggregate((previousAuthAverage - newAuthorityValue.getValue())); + } + setNewVertexValue(new Tuple2<>(newHubValue, newAuthorityValue)); + } else if (getSuperstepNumber() == maxIteration) { + + //final iteration to normalize hub score + newHubValue.setValue(newHubValue.getValue() / iterationValueSum); + setNewVertexValue(new Tuple2<>(newHubValue, newAuthorityValue)); + } + } + } + + /** + * Distributes the value of a vertex among all neighbor vertices and sum all the + * value in every superstep. + */ + @SuppressWarnings("serial") + public static final class MessageUpdate extends MessagingFunction, Double, Boolean> { + private int maxIteration; + + public MessageUpdate(int maxIteration) { + this.maxIteration = maxIteration; + } + + @Override + public void sendMessages(Vertex> vertex) { + + // in the first iteration, no aggregation to call, init sum with value of vertex + double iterationValueSum = 1.0; + + if (getSuperstepNumber() > 1) { + iterationValueSum = Math.sqrt(((DoubleValue) getPreviousIterationAggregate("updatedValueSum")).getValue()); + } + for (Edge edge : getEdges()) { + if (getSuperstepNumber() != maxIteration) { + if (getSuperstepNumber() % 2 == 1) { + if (edge.getValue()) { + sendMessageTo(edge.getTarget(), vertex.getValue().f0.getValue() / iterationValueSum); + } + } else { + if (!edge.getValue()) { + sendMessageTo(edge.getTarget(), vertex.getValue().f1.getValue() / iterationValueSum); + } + } + } else { + if (!edge.getValue()) { + sendMessageTo(edge.getTarget(), iterationValueSum); + } + } + } + } + } + + public static class VertexInitMapper implements MapFunction, Tuple2> { + + private static final long serialVersionUID = 1L; + + private Tuple2 initVertexValue = new Tuple2<>(new DoubleValue(1.0), new DoubleValue(1.0)); + + public Tuple2 map(Vertex value) { + + //init hub and authority value of each vertex + return initVertexValue; + } + } + + public static class AuthorityEdgeMapper implements MapFunction, Boolean> { + + private static final long serialVersionUID = 1L; + + public Boolean map(Edge edge) { + + // mark edge as true for authority updating + return true; + } + } + + public static class HubEdgeMapper implements MapFunction, Boolean> { + + private static final long serialVersionUID = 1L; + + public Boolean map(Edge edge) { + + // mark edge as false for hub updating + return false; + } + } +} + From 639eb74c4abf72311b312434ad4e82792000d4a6 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Sun, 22 May 2016 15:46:06 +0200 Subject: [PATCH 63/70] [FLINK-3936] [tableAPI] Add MIN/MAX aggregation for Boolean. This closes #2035 --- .../runtime/aggregate/AggregateUtil.scala | 4 +++ .../runtime/aggregate/MaxAggregate.scala | 15 ++++++++-- .../runtime/aggregate/MinAggregate.scala | 15 ++++++++-- .../runtime/aggregate/MaxAggregateTest.scala | 29 +++++++++++++++++++ .../runtime/aggregate/MinAggregateTest.scala | 29 +++++++++++++++++++ 5 files changed, 86 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala index 8222a2e2140e8..82364ebf6407d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala @@ -192,6 +192,8 @@ object AggregateUtil { new FloatMinAggregate case DOUBLE => new DoubleMinAggregate + case BOOLEAN => + new BooleanMinAggregate case sqlType: SqlTypeName => throw new TableException("Min aggregate does no support type:" + sqlType) } @@ -209,6 +211,8 @@ object AggregateUtil { new FloatMaxAggregate case DOUBLE => new DoubleMaxAggregate + case BOOLEAN => + new BooleanMaxAggregate case sqlType: SqlTypeName => throw new TableException("Max aggregate does no support type:" + sqlType) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala index 8f491f26c8d24..9ad0468bf1a9b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala @@ -20,9 +20,8 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.table.Row -abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { +abstract class MaxAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { - private val numeric = implicitly[Numeric[T]] protected var maxIndex = -1 /** @@ -49,7 +48,8 @@ abstract class MaxAggregate[T: Numeric] extends Aggregate[T] { override def merge(intermediate: Row, buffer: Row): Unit = { val partialValue = intermediate.productElement(maxIndex).asInstanceOf[T] val bufferValue = buffer.productElement(maxIndex).asInstanceOf[T] - buffer.setField(maxIndex, numeric.max(partialValue, bufferValue)) + val max: T = if (ord.compare(partialValue, bufferValue) > 0) partialValue else bufferValue + buffer.setField(maxIndex, max) } /** @@ -122,3 +122,12 @@ class DoubleMaxAggregate extends MaxAggregate[Double] { intermediate.setField(maxIndex, Double.MinValue) } } + +class BooleanMaxAggregate extends MaxAggregate[Boolean] { + + override def intermediateDataType = Array(BasicTypeInfo.BOOLEAN_TYPE_INFO) + + override def initiate(intermediate: Row): Unit = { + intermediate.setField(maxIndex, false) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala index e78fb00cccd35..b607e6b4bc1d6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala @@ -20,9 +20,8 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.table.Row -abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ +abstract class MinAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T]{ - private val numeric = implicitly[Numeric[T]] protected var minIndex: Int = _ /** @@ -49,7 +48,8 @@ abstract class MinAggregate[T: Numeric] extends Aggregate[T]{ override def merge(partial: Row, buffer: Row): Unit = { val partialValue = partial.productElement(minIndex).asInstanceOf[T] val bufferValue = buffer.productElement(minIndex).asInstanceOf[T] - buffer.setField(minIndex, numeric.min(partialValue, bufferValue)) + val min: T = if (ord.compare(partialValue, bufferValue) < 0) partialValue else bufferValue + buffer.setField(minIndex, min) } /** @@ -122,3 +122,12 @@ class DoubleMinAggregate extends MinAggregate[Double] { intermediate.setField(minIndex, Double.MaxValue) } } + +class BooleanMinAggregate extends MinAggregate[Boolean] { + + override def intermediateDataType = Array(BasicTypeInfo.BOOLEAN_TYPE_INFO) + + override def initiate(intermediate: Row): Unit = { + intermediate.setField(minIndex, true) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala index f3951e4dc3618..e049e49916d6d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala @@ -91,3 +91,32 @@ class DoubleMaxAggregateTest extends MaxAggregateTestBase[Double] { override def aggregator: Aggregate[Double] = new DoubleMaxAggregate() } + +class BooleanMaxAggregateTest extends AggregateTestBase[Boolean] { + + override def inputValueSets: Seq[Seq[Boolean]] = Seq( + Seq( + false, + false, + false + ), + Seq( + true, + true, + true + ), + Seq( + true, + false, + null.asInstanceOf[Boolean], + true, + false, + true, + null.asInstanceOf[Boolean] + ) + ) + + override def expectedResults: Seq[Boolean] = Seq(false, true, true) + + override def aggregator: Aggregate[Boolean] = new BooleanMaxAggregate() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala index 3a4b111d53fff..7cf7bb182889f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala @@ -91,3 +91,32 @@ class DoubleMinAggregateTest extends MinAggregateTestBase[Double] { override def aggregator: Aggregate[Double] = new DoubleMinAggregate() } + +class BooleanMinAggregateTest extends AggregateTestBase[Boolean] { + + override def inputValueSets: Seq[Seq[Boolean]] = Seq( + Seq( + false, + false, + false + ), + Seq( + true, + true, + true + ), + Seq( + true, + false, + null.asInstanceOf[Boolean], + true, + false, + true, + null.asInstanceOf[Boolean] + ) + ) + + override def expectedResults: Seq[Boolean] = Seq(false, true, false) + + override def aggregator: Aggregate[Boolean] = new BooleanMinAggregate() +} From 1307f954004470c5d697832019e5c478daab9e46 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Tue, 24 May 2016 12:46:21 +0800 Subject: [PATCH 64/70] [FLINK-3941] [tableAPI] Add support for UNION to Table API. - Fix FLINK-3696 (type issues of DataSetUnion by forwarding expected types to input operators). This closes #2025 --- docs/apis/table.md | 24 +++ .../api/table/plan/logical/operators.scala | 4 +- .../plan/nodes/dataset/DataSetUnion.scala | 17 +- .../plan/rules/dataSet/DataSetUnionRule.scala | 5 +- .../org/apache/flink/api/table/table.scala | 25 ++- .../api/java/batch/table/UnionITCase.java | 186 ------------------ .../api/scala/batch/sql/UnionITCase.scala | 32 +-- .../api/scala/batch/table/UnionITCase.scala | 78 +++----- .../stream/table/UnsupportedOpsTest.scala | 9 + 9 files changed, 121 insertions(+), 259 deletions(-) delete mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java diff --git a/docs/apis/table.md b/docs/apis/table.md index 4e52a983500ed..f1e9cd1fe2268 100644 --- a/docs/apis/table.md +++ b/docs/apis/table.md @@ -434,6 +434,18 @@ Table result = left.join(right).where("a = d").select("a, b, e"); Union + +

Similar to a SQL UNION clause. Unions two tables with duplicate records removed. Both tables must have identical schema, i.e., field names and types.

+{% highlight java %} +Table left = tableEnv.fromDataSet(ds1, "a, b, c"); +Table right = tableEnv.fromDataSet(ds2, "a, b, c"); +Table result = left.union(right); +{% endhighlight %} + + + + + UnionAll

Similar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical schema, i.e., field names and types.

{% highlight java %} @@ -545,6 +557,18 @@ val result = left.join(right).where('a === 'd).select('a, 'b, 'e); Union + +

Similar to a SQL UNION clause. Unions two tables with duplicate records removed, both tables must have identical schema(field names and types).

+{% highlight scala %} +val left = ds1.toTable(tableEnv, 'a, 'b, 'c); +val right = ds2.toTable(tableEnv, 'a, 'b, 'c); +val result = left.union(right); +{% endhighlight %} + + + + + UnionAll

Similar to a SQL UNION ALL clause. Unions two tables, both tables must have identical schema(field names and types).

{% highlight scala %} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala index bd299b36ab94e..6b42a7d508c63 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala @@ -236,13 +236,13 @@ case class Aggregate( } } -case class Union(left: LogicalNode, right: LogicalNode) extends BinaryNode { +case class Union(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode { override def output: Seq[Attribute] = left.output override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = { left.construct(relBuilder) right.construct(relBuilder) - relBuilder.union(true) + relBuilder.union(all) } override def validate(tableEnv: TableEnvironment): LogicalNode = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala index b6f6a19ccd5f7..78f64a464a848 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala @@ -55,7 +55,7 @@ class DataSetUnion( } override def toString: String = { - "Union(union: (${rowType.getFieldNames.asScala.toList.mkString(\", \")}))" + s"Union(union: (${rowType.getFieldNames.asScala.toList.mkString(", ")}))" } override def explainTerms(pw: RelWriter): RelWriter = { @@ -76,8 +76,19 @@ class DataSetUnion( tableEnv: BatchTableEnvironment, expectedType: Option[TypeInformation[Any]]): DataSet[Any] = { - val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv) - val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv) + var leftDataSet: DataSet[Any] = null + var rightDataSet: DataSet[Any] = null + + expectedType match { + case None => + leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv) + rightDataSet = + right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, Some(leftDataSet.getType)) + case _ => + leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv, expectedType) + rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, expectedType) + } + leftDataSet.union(rightDataSet).asInstanceOf[DataSet[Any]] } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala index 7809d6d1ac1ec..ea35637efe4dc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala @@ -22,6 +22,7 @@ import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTrait import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.logical.LogicalUnion +import org.apache.calcite.rel.rules.UnionToDistinctRule import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetUnion} class DataSetUnionRule @@ -33,7 +34,9 @@ class DataSetUnionRule { /** - * Only translate UNION ALL + * Only translate UNION ALL. + * Note: A distinct Union are translated into + * an Aggregate on top of a UNION ALL by [[UnionToDistinctRule]] */ override def matches(call: RelOptRuleCall): Boolean = { val union: LogicalUnion = call.rel(0).asInstanceOf[LogicalUnion] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala index 1e558c5448c2d..394b833dfc493 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala @@ -259,6 +259,29 @@ class Table( Join(this.logicalPlan, right.logicalPlan, JoinType.INNER, None).validate(tableEnv)) } + /** + * Union two [[Table]]s with duplicate records removed. + * Similar to an SQL UNION. The fields of the two union operations must fully overlap. + * + * Note: Both tables must be bound to the same [[TableEnvironment]]. + * + * Example: + * + * {{{ + * left.union(right) + * }}} + */ + def union(right: Table): Table = { + if (tableEnv.isInstanceOf[StreamTableEnvironment]) { + throw new TableException(s"Union on stream tables is currently not supported.") + } + // check that right table belongs to the same TableEnvironment + if (right.tableEnv != this.tableEnv) { + throw new ValidationException("Only tables from the same TableEnvironment can be unioned.") + } + new Table(tableEnv, Union(logicalPlan, right.logicalPlan, false).validate(tableEnv)) + } + /** * Union two [[Table]]s. Similar to an SQL UNION ALL. The fields of the two union operations * must fully overlap. @@ -276,7 +299,7 @@ class Table( if (right.tableEnv != this.tableEnv) { throw new ValidationException("Only tables from the same TableEnvironment can be unioned.") } - new Table(tableEnv, Union(logicalPlan, right.logicalPlan).validate(tableEnv)) + new Table(tableEnv, Union(logicalPlan, right.logicalPlan, true).validate(tableEnv)) } /** diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java deleted file mode 100644 index 853cd7f5403ce..0000000000000 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/UnionITCase.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.java.batch.table; - -import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.table.BatchTableEnvironment; -import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.TableEnvironment; -import org.apache.flink.api.table.ValidationException; -import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.apache.flink.test.util.MultipleProgramsTestBase; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.List; - -@RunWith(Parameterized.class) -public class UnionITCase extends MultipleProgramsTestBase { - - public UnionITCase(TestExecutionMode mode) { - super(mode); - } - - @Test - public void testUnion() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "a, b, c"); - - Table selected = in1.unionAll(in2).select("c"); - - DataSet ds = tableEnv.toDataSet(selected, Row.class); - List results = ds.collect(); - String expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hi\n" + "Hello\n" + "Hello world\n"; - compareResultAsText(results, expected); - } - - @Test - public void testUnionWithFilter() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "a, b, d, c, e").select("a, b, c"); - - Table selected = in1.unionAll(in2).where("b < 2").select("c"); - - DataSet ds = tableEnv.toDataSet(selected, Row.class); - List results = ds.collect(); - String expected = "Hi\n" + "Hallo\n"; - compareResultAsText(results, expected); - } - - @Test(expected = ValidationException.class) - public void testUnionIncompatibleNumberOfFields() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h"); - - // Must fail. Number of fields of union inputs do not match - in1.unionAll(in2); - } - - @Test(expected = ValidationException.class) - public void testUnionIncompatibleFieldsName() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "a, b, d"); - - // Must fail. Field names of union inputs do not match - in1.unionAll(in2); - } - - @Test(expected = ValidationException.class) - public void testUnionIncompatibleFieldTypes() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "a, b, c, d, e").select("a, b, c"); - - // Must fail. Field types of union inputs do not match - in1.unionAll(in2); - } - - @Test - public void testUnionWithAggregation() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "a, b, d, c, e").select("a, b, c"); - - Table selected = in1.unionAll(in2).select("c.count"); - - DataSet ds = tableEnv.toDataSet(selected, Row.class); - List results = ds.collect(); - String expected = "18"; - compareResultAsText(results, expected); - } - - @Test - public void testUnionWithJoin() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); - DataSet> ds3 = CollectionDataSets.getSmall5TupleDataSet(env); - - Table in1 = tableEnv.fromDataSet(ds1, "a, b, c"); - Table in2 = tableEnv.fromDataSet(ds2, "a, b, d, c, e").select("a, b, c"); - Table in3 = tableEnv.fromDataSet(ds3, "a2, b2, d2, c2, e2").select("a2, b2, c2"); - - Table joinDs = in1.unionAll(in2).join(in3).where("a === a2").select("c, c2"); - DataSet ds = tableEnv.toDataSet(joinDs, Row.class); - List results = ds.collect(); - - String expected = "Hi,Hallo\n" + "Hallo,Hallo\n" + - "Hello,Hallo Welt\n" + "Hello,Hallo Welt wie\n" + - "Hallo Welt,Hallo Welt\n" + "Hallo Welt wie,Hallo Welt\n" + - "Hallo Welt,Hallo Welt wie\n" + "Hallo Welt wie,Hallo Welt wie\n"; - compareResultAsText(results, expected); - } - - @Test(expected = ValidationException.class) - public void testUnionTablesFromDifferentEnvs() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - BatchTableEnvironment tEnv1 = TableEnvironment.getTableEnvironment(env); - BatchTableEnvironment tEnv2 = TableEnvironment.getTableEnvironment(env); - - DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); - DataSet> ds2 = CollectionDataSets.getSmall3TupleDataSet(env); - - Table in1 = tEnv1.fromDataSet(ds1, "a, b, c"); - Table in2 = tEnv2.fromDataSet(ds2, "a, b, c"); - - // Must fail. Tables are bound to different TableEnvironments. - in1.unionAll(in2); - } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UnionITCase.scala index a42d3283835d9..527eac75c592d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UnionITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UnionITCase.scala @@ -39,7 +39,7 @@ class UnionITCase( extends TableProgramsTestBase(mode, configMode) { @Test - def testUnion(): Unit = { + def testUnionAll(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -58,16 +58,31 @@ class UnionITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - //TODO: activate for EFFICIENT mode @Test - def testUnionWithFilter(): Unit = { + def testUnion(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) - if (tEnv.getConfig.getEfficientTypeUsage) { - return - } + val sqlQuery = "SELECT c FROM t1 UNION (SELECT c FROM t2)" + + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) + val ds2 = CollectionDataSets.getSmall3TupleDataSet(env) + tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) + tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c) + + val result = tEnv.sql(sqlQuery) + + val expected = "Hi\n" + "Hello\n" + "Hello world\n" + val results = result.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testUnionWithFilter(): Unit = { + + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) val sqlQuery = "SELECT c FROM (" + "SELECT * FROM t1 UNION ALL (SELECT a, b, c FROM t2))" + @@ -85,17 +100,12 @@ class UnionITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } - //TODO: activate for EFFICIENT mode @Test def testUnionWithAggregation(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) - if (tEnv.getConfig.getEfficientTypeUsage) { - return - } - val sqlQuery = "SELECT count(c) FROM (" + "SELECT * FROM t1 UNION ALL (SELECT a, b, c FROM t2))" diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala index 29427a5d29caa..f472341284bc9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UnionITCase.scala @@ -39,7 +39,7 @@ class UnionITCase( extends TableProgramsTestBase(mode, configMode) { @Test - def testUnion(): Unit = { + def testUnionAll(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -54,7 +54,22 @@ class UnionITCase( } @Test - def testTernaryUnion(): Unit = { + def testUnion(): Unit = { + val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + + val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + + val unionDs = ds1.union(ds2).select('c) + + val results = unionDs.toDataSet[Row].collect() + val expected = "Hi\n" + "Hello\n" + "Hello world\n" + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + + @Test + def testTernaryUnionAll(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) @@ -72,21 +87,18 @@ class UnionITCase( } @Test - def testUnionWithFilter(): Unit = { + def testTernaryUnion(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) - if (tEnv.getConfig.getEfficientTypeUsage) { - return - } - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e) + val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) + val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - val joinDs = ds1.unionAll(ds2.select('a, 'b, 'c)).filter('b < 2).select('c) + val unionDs = ds1.union(ds2).union(ds3).select('c) - val results = joinDs.toDataSet[Row].collect() - val expected = "Hi\n" + "Hallo\n" + val results = unionDs.toDataSet[Row].collect() + val expected = "Hi\n" + "Hello\n" + "Hello world\n" TestBaseUtils.compareResultAsText(results.asJava, expected) } @@ -115,50 +127,6 @@ class UnionITCase( ds1.unionAll(ds2) } - @Test - def testUnionWithAggregation(): Unit = { - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - if (tEnv.getConfig.getEfficientTypeUsage) { - return - } - - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e) - - val unionDs = ds1.unionAll(ds2.select('a, 'b, 'c)).select('c.count) - - val results = unionDs.toDataSet[Row].collect() - val expected = "18" - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - - @Test - def testUnionWithJoin(): Unit = { - val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment - val tEnv = TableEnvironment.getTableEnvironment(env, config) - - if (tEnv.getConfig.getEfficientTypeUsage) { - return - } - - val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) - val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv,'a, 'b, 'd, 'c, 'e) - val ds3 = CollectionDataSets.getSmall5TupleDataSet(env).toTable(tEnv, 'a2, 'b2, 'd2, 'c2, 'e2) - - val joinDs = ds1.unionAll(ds2.select('a, 'b, 'c)) - .join(ds3.select('a2, 'b2, 'c2)) - .where('a ==='a2).select('c, 'c2) - - val results = joinDs.toDataSet[Row].collect() - val expected = "Hi,Hallo\n" + "Hallo,Hallo\n" + - "Hello,Hallo Welt\n" + "Hello,Hallo Welt wie\n" + - "Hallo Welt,Hallo Welt\n" + "Hallo Welt wie,Hallo Welt\n" + - "Hallo Welt,Hallo Welt wie\n" + "Hallo Welt wie,Hallo Welt wie\n" - TestBaseUtils.compareResultAsText(results.asJava, expected) - } - @Test(expected = classOf[ValidationException]) def testUnionTablesFromDifferentEnvs(): Unit = { val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala index a382447bcb9d0..92de6f17fa35d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala @@ -64,4 +64,13 @@ class UnsupportedOpsTest extends StreamingMultipleProgramsTestBase { val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv) t1.join(t2) } + + @Test(expected = classOf[TableException]) + def testUnion(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv) + val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv) + t1.union(t2) + } } From ef135c3dbe2ceda58c53fd6ea77960861eb2f899 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 25 May 2016 16:20:27 +0200 Subject: [PATCH 65/70] [hotfix] Remove leftover config key constant from ExecutionConfig --- .../main/java/org/apache/flink/api/common/ExecutionConfig.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index d27760f2f4a77..86d3be609c56e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -61,9 +61,6 @@ public class ExecutionConfig implements Serializable { private static final long serialVersionUID = 1L; - // Key for storing it in the Job Configuration - public static final String CONFIG_KEY = "runtime.config"; - /** * The constant to use for the parallelism, if the system should use the number * of currently available slots. From 030b4f8c0fb606dfe2a1598ec5e116e8543284c8 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 25 May 2016 17:04:31 +0200 Subject: [PATCH 66/70] [hotfix] Fix access to temp file directories in SpillingAdaptiveSpanningRecordDeserializer --- .../api/common/functions/RuntimeContext.java | 2 +- .../org/apache/flink/metrics/MetricGroup.java | 6 +-- .../clusterframework/types/ResourceID.java | 34 +++++++------- .../runtime/io/disk/iomanager/IOManager.java | 15 ++++++- .../api/reader/AbstractRecordReader.java | 12 ++++- .../api/reader/MutableRecordReader.java | 12 ++++- .../io/network/api/reader/RecordReader.java | 17 ++++--- ...ingAdaptiveSpanningRecordDeserializer.java | 16 ++----- .../IterationSynchronizationSinkTask.java | 6 ++- .../flink/runtime/operators/BatchTask.java | 18 +++++--- .../flink/runtime/operators/DataSinkTask.java | 8 +++- .../taskmanager/TaskManagerRuntimeInfo.java | 35 +++++++++++++-- .../runtime/taskmanager/TaskManager.scala | 3 +- .../SpanningRecordSerializationTest.java | 4 +- .../serialization/LargeRecordsTest.java | 5 ++- .../SlotCountExceedingParallelismTest.java | 5 ++- .../operators/drivers/TestTaskContext.java | 3 +- .../testutils/BinaryOperatorTestBase.java | 3 +- .../operators/testutils/DriverTestBase.java | 3 +- .../operators/testutils/MockEnvironment.java | 5 ++- .../testutils/UnaryOperatorTestBase.java | 3 +- .../taskmanager/TaskAsyncCallTest.java | 2 +- .../runtime/taskmanager/TaskCancelTest.java | 10 +++-- .../flink/runtime/taskmanager/TaskTest.java | 2 +- .../flink/runtime/jobmanager/Tasks.scala | 44 +++++++++++++++---- .../runtime/io/StreamInputProcessor.java | 3 +- .../runtime/io/StreamTwoInputProcessor.java | 3 +- .../runtime/tasks/StreamMockEnvironment.java | 4 +- .../runtime/tasks/StreamTaskTest.java | 2 +- .../runtime/NetworkStackThroughputITCase.java | 20 ++++++--- 30 files changed, 212 insertions(+), 93 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index d623707187544..382107fae0f08 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -64,7 +64,7 @@ public interface RuntimeContext { * Returns the metric group for this parallel subtask. * * @return The metric group for this parallel subtask. - */ + */ MetricGroup getMetricGroup(); /** diff --git a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java index a3832ff09f77e..6c9e04477acd2 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/MetricGroup.java @@ -21,14 +21,14 @@ import org.apache.flink.annotation.PublicEvolving; /** - * A MetricGroup is a named container for {@link Metric Metrics} and {@link MetricGroup MetricGroups}. + * A MetricGroup is a named container for {@link Metric Metrics} and further metric subgroups. * *

Instances of this class can be used to register new metrics with Flink and to create a nested * hierarchy based on the group names. * *

A MetricGroup is uniquely identified by it's place in the hierarchy and name. * - *

Metrics groups can be {@link #close() closed}. Upon closing, they de-register all metrics + *

Metrics groups can be {@link #close() closed}. Upon closing, the group de-register all metrics * from any metrics reporter and any internal maps. Note that even closed metrics groups * return Counters, Gauges, etc to the code, to prevent exceptions in the monitored code. * These metrics simply do not get reported any more, when created on a closed group. @@ -39,7 +39,7 @@ public interface MetricGroup { // ------------------------------------------------------------------------ // Closing // ------------------------------------------------------------------------ - + /** * Marks the group as closed. * Recursively unregisters all {@link Metric Metrics} contained in this group. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java index e599456bcbe09..9d82c76325c7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java @@ -45,23 +45,11 @@ public final String getResourceIdString() { return resourceId; } - /** - * Generate a random resource id. - * @return A random resource id. - */ - public static ResourceID generate() { - return new ResourceID(new AbstractID().toString()); - } - @Override public final boolean equals(Object o) { - if (this == o) { - return true; - } else if (o == null || !(o instanceof ResourceID)) { - return false; - } else { - return resourceId.equals(((ResourceID) o).resourceId); - } + return this == o || + (o != null && o.getClass() == ResourceID.class && + this.resourceId.equals(((ResourceID) o).resourceId)); } @Override @@ -71,8 +59,18 @@ public final int hashCode() { @Override public String toString() { - return "ResourceID{" + - "resourceId='" + resourceId + '\'' + - '}'; + return "ResourceID (" + resourceId + ')'; + } + + // ------------------------------------------------------------------------ + // factory + // ------------------------------------------------------------------------ + + /** + * Generate a random resource id. + * @return A random resource id. + */ + public static ResourceID generate() { + return new ResourceID(new AbstractID().toString()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java index 0942f722aebe8..7904cc4e5b3c4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java @@ -21,6 +21,7 @@ import org.apache.commons.io.FileUtils; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.network.buffer.Buffer; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -281,7 +282,19 @@ public int getNumberOfSpillingDirectories() { public File[] getSpillingDirectories() { return this.paths; } - + + /** + * Gets the directories that the I/O manager spills to, as path strings. + * + * @return The directories that the I/O manager spills to, as path strings. + */ + public String[] getSpillingDirectoriesPaths() { + String[] strings = new String[this.paths.length]; + for (int i = 0; i < strings.length; i++) { + strings[i] = paths[i].getAbsolutePath(); + } + return strings; + } protected int getNextPathNum() { final int next = this.nextPath; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java index a784f5444ca7d..48ac558daa55d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/AbstractRecordReader.java @@ -45,14 +45,22 @@ abstract class AbstractRecordReader extends Abstra private boolean isFinished; + /** + * Creates a new AbstractRecordReader that de-serializes records from the given input gate and + * can spill partial records to disk, if they grow large. + * + * @param inputGate The input gate to read from. + * @param tmpDirectories The temp directories. USed for spilling if the reader concurrently + * reconstructs multiple large records. + */ @SuppressWarnings("unchecked") - protected AbstractRecordReader(InputGate inputGate) { + protected AbstractRecordReader(InputGate inputGate, String[] tmpDirectories) { super(inputGate); // Initialize one deserializer per input channel this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; for (int i = 0; i < recordDeserializers.length; i++) { - recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(); + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer(tmpDirectories); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/MutableRecordReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/MutableRecordReader.java index d7cc7e92821f5..9836ba465ea8b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/MutableRecordReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/MutableRecordReader.java @@ -25,8 +25,16 @@ public class MutableRecordReader extends AbstractRecordReader implements MutableReader { - public MutableRecordReader(InputGate inputGate) { - super(inputGate); + /** + * Creates a new MutableRecordReader that de-serializes records from the given input gate and + * can spill partial records to disk, if they grow large. + * + * @param inputGate The input gate to read from. + * @param tmpDirectories The temp directories. USed for spilling if the reader concurrently + * reconstructs multiple large records. + */ + public MutableRecordReader(InputGate inputGate, String[] tmpDirectories) { + super(inputGate, tmpDirectories); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/RecordReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/RecordReader.java index d45920e646481..9eed3740ff087 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/RecordReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/RecordReader.java @@ -29,8 +29,16 @@ public class RecordReader extends AbstractRecordRe private T currentRecord; - public RecordReader(InputGate inputGate, Class recordType) { - super(inputGate); + /** + * Creates a new RecordReader that de-serializes records from the given input gate and + * can spill partial records to disk, if they grow large. + * + * @param inputGate The input gate to read from. + * @param tmpDirectories The temp directories. USed for spilling if the reader concurrently + * reconstructs multiple large records. + */ + public RecordReader(InputGate inputGate, Class recordType, String[] tmpDirectories) { + super(inputGate, tmpDirectories); this.recordType = recordType; } @@ -73,10 +81,7 @@ private T instantiateRecordType() { try { return recordType.newInstance(); } - catch (InstantiationException e) { - throw new RuntimeException("Cannot instantiate class " + recordType.getName(), e); - } - catch (IllegalAccessException e) { + catch (InstantiationException | IllegalAccessException e) { throw new RuntimeException("Cannot instantiate class " + recordType.getName(), e); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java index 49f7584497c85..7e96390c9d9a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.io.network.api.serialization; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; @@ -65,18 +63,12 @@ public class SpillingAdaptiveSpanningRecordDeserializer(getEnvironment().getInputGate(0)); + this.headEventReader = new MutableRecordReader( + getEnvironment().getInputGate(0), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); TaskConfig taskConfig = new TaskConfig(getTaskConfiguration()); @@ -184,7 +186,7 @@ private void readHeadEventChannel(IntValue rec) throws IOException { // read (and thereby process all events in the handler's event handling functions) try { - while (this.headEventReader.next(rec)) { + if (this.headEventReader.next(rec)) { throw new RuntimeException("Synchronization task must not see any records!"); } } catch (InterruptedException iex) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java index 546193c4892d6..f38b988975db3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java @@ -659,14 +659,18 @@ protected void initInputReaders() throws Exception { if (groupSize == 1) { // non-union case - inputReaders[i] = new MutableRecordReader(getEnvironment().getInputGate(currentReaderOffset)); + inputReaders[i] = new MutableRecordReader( + getEnvironment().getInputGate(currentReaderOffset), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else if (groupSize > 1){ // union case InputGate[] readers = new InputGate[groupSize]; for (int j = 0; j < groupSize; ++j) { readers[j] = getEnvironment().getInputGate(currentReaderOffset + j); } - inputReaders[i] = new MutableRecordReader(new UnionInputGate(readers)); + inputReaders[i] = new MutableRecordReader( + new UnionInputGate(readers), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else { throw new Exception("Illegal input group size in task configuration: " + groupSize); } @@ -701,14 +705,18 @@ protected void initBroadcastInputReaders() throws Exception { final int groupSize = this.config.getBroadcastGroupSize(i); if (groupSize == 1) { // non-union case - broadcastInputReaders[i] = new MutableRecordReader(getEnvironment().getInputGate(currentReaderOffset)); + broadcastInputReaders[i] = new MutableRecordReader( + getEnvironment().getInputGate(currentReaderOffset), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else if (groupSize > 1){ // union case InputGate[] readers = new InputGate[groupSize]; for (int j = 0; j < groupSize; ++j) { readers[j] = getEnvironment().getInputGate(currentReaderOffset + j); } - broadcastInputReaders[i] = new MutableRecordReader(new UnionInputGate(readers)); + broadcastInputReaders[i] = new MutableRecordReader( + new UnionInputGate(readers), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else { throw new Exception("Illegal input group size in task configuration: " + groupSize); } @@ -765,8 +773,6 @@ protected void initBroadcastInputsSerializers(int numBroadcastInputs) throws Exc * * NOTE: This method must be invoked after the invocation of {@code #initInputReaders()} and * {@code #initInputSerializersAndComparators(int)}! - * - * @param numInputs */ protected void initLocalStrategies(int numInputs) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java index 39bf23f126167..380edd4cb9aed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSinkTask.java @@ -332,10 +332,14 @@ private void initInputReaders() throws Exception { numGates += groupSize; if (groupSize == 1) { // non-union case - inputReader = new MutableRecordReader>(getEnvironment().getInputGate(0)); + inputReader = new MutableRecordReader>( + getEnvironment().getInputGate(0), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else if (groupSize > 1){ // union case - inputReader = new MutableRecordReader(new UnionInputGate(getEnvironment().getAllInputGates())); + inputReader = new MutableRecordReader( + new UnionInputGate(getEnvironment().getAllInputGates()), + getEnvironment().getTaskManagerInfo().getTmpDirectories()); } else { throw new Exception("Illegal input group size in task configuration: " + groupSize); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java index 8d06f1006c63c..9ac982eba3dd7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRuntimeInfo.java @@ -20,6 +20,9 @@ import org.apache.flink.configuration.Configuration; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * Encapsulation of TaskManager runtime information, like hostname and configuration. */ @@ -33,14 +36,32 @@ public class TaskManagerRuntimeInfo implements java.io.Serializable { /** configuration that the TaskManager was started with */ private final Configuration configuration; + /** list of temporary file directories */ + private final String[] tmpDirectories; + /** * Creates a runtime info. + * * @param hostname The host name of the interface that the TaskManager uses to communicate. * @param configuration The configuration that the TaskManager was started with. + * @param tmpDirectory The temporary file directory. */ - public TaskManagerRuntimeInfo(String hostname, Configuration configuration) { - this.hostname = hostname; - this.configuration = configuration; + public TaskManagerRuntimeInfo(String hostname, Configuration configuration, String tmpDirectory) { + this(hostname, configuration, new String[] { tmpDirectory }); + } + + /** + * Creates a runtime info. + * @param hostname The host name of the interface that the TaskManager uses to communicate. + * @param configuration The configuration that the TaskManager was started with. + * @param tmpDirectories The list of temporary file directories. + */ + public TaskManagerRuntimeInfo(String hostname, Configuration configuration, String[] tmpDirectories) { + checkArgument(tmpDirectories.length > 0); + this.hostname = checkNotNull(hostname); + this.configuration = checkNotNull(configuration); + this.tmpDirectories = tmpDirectories; + } /** @@ -58,4 +79,12 @@ public String getHostname() { public Configuration getConfiguration() { return configuration; } + + /** + * Gets the list of temporary file directories. + * @return The list of temporary file directories. + */ + public String[] getTmpDirectories() { + return tmpDirectories; + } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index a5cc18d5622cd..eb7a0ef6ec48f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -187,7 +187,8 @@ class TaskManager( private val runtimeInfo = new TaskManagerRuntimeInfo( connectionInfo.getHostname(), - new UnmodifiableConfiguration(config.configuration)) + new UnmodifiableConfiguration(config.configuration), + config.tmpDirPaths) // -------------------------------------------------------------------------- // Actor messages and life cycle // -------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java index 819a94ff735c0..9d0ee67134f04 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java @@ -106,7 +106,9 @@ private void testNonSpillingDeserializer(Util.MockRecords records, int segmentSi private void testSpillingDeserializer(Util.MockRecords records, int segmentSize) throws Exception { RecordSerializer serializer = new SpanningRecordSerializer(); - RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer(); + RecordDeserializer deserializer = + new SpillingAdaptiveSpanningRecordDeserializer( + new String[] { System.getProperty("java.io.tmpdir") }); test(records, segmentSize, serializer, deserializer); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java index d628596e6ea6b..1574fe95a372a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java @@ -147,7 +147,10 @@ public void testHandleMixedLargeRecordsSpillingAdaptiveSerializer() { final int SEGMENT_SIZE = 32 * 1024; final RecordSerializer serializer = new SpanningRecordSerializer(); - final RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer(); + + final RecordDeserializer deserializer = + new SpillingAdaptiveSpanningRecordDeserializer( + new String[] { System.getProperty("java.io.tmpdir") } ); final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java index 561bda304f805..e12faf973ff5a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotCountExceedingParallelismTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmanager; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.io.network.api.reader.RecordReader; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; @@ -30,6 +29,7 @@ import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.types.IntValue; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -152,7 +152,8 @@ public static class SubtaskIndexReceiver extends AbstractInvokable { public void invoke() throws Exception { RecordReader reader = new RecordReader<>( getEnvironment().getInputGate(0), - IntValue.class); + IntValue.class, + getEnvironment().getTaskManagerInfo().getTmpDirectories()); try { final int numberOfSubtaskIndexesToReceive = getTaskConfiguration().getInteger(CONFIG_KEY, 0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index 0300a079b1f39..15ad353de497f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -74,7 +74,8 @@ public TestTaskContext() {} public TestTaskContext(long memoryInBytes) { this.memoryManager = new MemoryManager(memoryInBytes, 1, 32 * 1024, MemoryType.HEAP, true); - this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); + this.taskManageInfo = new TaskManagerRuntimeInfo( + "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java index 2c3dcf144ede2..6e9b81719a50f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java @@ -110,7 +110,8 @@ protected BinaryOperatorTestBase(ExecutionConfig executionConfig, long memory, i this.owner = new DummyInvokable(); this.taskConfig = new TaskConfig(new Configuration()); this.executionConfig = executionConfig; - this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); + this.taskManageInfo = new TaskManagerRuntimeInfo( + "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); } @Parameterized.Parameters diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 638173359c8db..eb2a3a736b1ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -114,7 +114,8 @@ protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNu this.owner = new DummyInvokable(); this.taskConfig = new TaskConfig(new Configuration()); this.executionConfig = executionConfig; - this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); + this.taskManageInfo = new TaskManagerRuntimeInfo( + "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); } @Parameterized.Parameters diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 31fd08c2957f6..b774b484d88d1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -208,7 +208,10 @@ public Configuration getJobConfiguration() { @Override public TaskManagerRuntimeInfo getTaskManagerInfo() { - return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); + return new TaskManagerRuntimeInfo( + "localhost", + new UnmodifiableConfiguration(new Configuration()), + System.getProperty("java.io.tmpdir")); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 77e18c60eac2a..50bb1ee3c3353 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -115,7 +115,8 @@ protected UnaryOperatorTestBase(ExecutionConfig executionConfig, long memory, in this.executionConfig = executionConfig; this.comparators = new ArrayList>(2); - this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration()); + this.taskManageInfo = new TaskManagerRuntimeInfo( + "localhost", new Configuration(), System.getProperty("java.io.tmpdir")); } @Parameterized.Parameters diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index ae05ae9e7a9cf..e1f551c0ee06f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -171,7 +171,7 @@ private static Task createTask() { new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration()), + new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), mock(TaskMetricGroup.class)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java index 09dd8170c262d..fc5a4a3fcb859 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.taskmanager; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -41,7 +40,9 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; import org.apache.flink.types.IntValue; + import org.junit.Test; + import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -247,10 +248,11 @@ public static class AgnosticUnion extends AbstractInvokable { @Override public void invoke() throws Exception { UnionInputGate union = new UnionInputGate(getEnvironment().getAllInputGates()); - RecordReader reader = new RecordReader<>(union, IntValue.class); + RecordReader reader = new RecordReader<>( + union, IntValue.class, getEnvironment().getTaskManagerInfo().getTmpDirectories()); - while (reader.next() != null) { - } + //noinspection StatementWithEmptyBody + while (reader.next() != null) {} } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index f237c877aaea9..1762a7ad6147a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -623,7 +623,7 @@ private Task createTask(Class invokable, new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration()), + new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), mock(TaskMetricGroup.class)); } diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala index d871c3df1294d..87c123aaab7a5 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/Tasks.scala @@ -64,7 +64,11 @@ object Tasks { class Forwarder extends AbstractInvokable { override def invoke(): Unit = { - val reader = new RecordReader[IntValue](getEnvironment.getInputGate(0), classOf[IntValue]) + val reader = new RecordReader[IntValue]( + getEnvironment.getInputGate(0), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) + val writer = new RecordWriter[IntValue](getEnvironment.getWriter(0)) try { @@ -88,7 +92,10 @@ object Tasks { class Receiver extends AbstractInvokable { override def invoke(): Unit = { - val reader = new RecordReader[IntValue](getEnvironment.getInputGate(0), classOf[IntValue]) + val reader = new RecordReader[IntValue]( + getEnvironment.getInputGate(0), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) val i1 = reader.next() val i2 = reader.next() @@ -140,7 +147,10 @@ object Tasks { class AgnosticReceiver extends AbstractInvokable { override def invoke(): Unit = { - val reader= new RecordReader[IntValue](getEnvironment.getInputGate(0), classOf[IntValue]) + val reader= new RecordReader[IntValue]( + getEnvironment.getInputGate(0), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) while(reader.next() != null){} } @@ -149,8 +159,15 @@ object Tasks { class AgnosticBinaryReceiver extends AbstractInvokable { override def invoke(): Unit = { - val reader1 = new RecordReader[IntValue](getEnvironment.getInputGate(0), classOf[IntValue]) - val reader2 = new RecordReader[IntValue](getEnvironment.getInputGate(1), classOf[IntValue]) + val reader1 = new RecordReader[IntValue]( + getEnvironment.getInputGate(0), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) + + val reader2 = new RecordReader[IntValue]( + getEnvironment.getInputGate(1), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) while(reader1.next() != null){} while(reader2.next() != null){} @@ -162,9 +179,20 @@ object Tasks { override def invoke(): Unit = { val env = getEnvironment - val reader1 = new RecordReader[IntValue](env.getInputGate(0), classOf[IntValue]) - val reader2 = new RecordReader[IntValue](env.getInputGate(1), classOf[IntValue]) - val reader3 = new RecordReader[IntValue](env.getInputGate(2), classOf[IntValue]) + val reader1 = new RecordReader[IntValue]( + env.getInputGate(0), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) + + val reader2 = new RecordReader[IntValue]( + env.getInputGate(1), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) + + val reader3 = new RecordReader[IntValue]( + env.getInputGate(2), + classOf[IntValue], + getEnvironment.getTaskManagerInfo.getTmpDirectories) while(reader1.next() != null){} while(reader2.next() != null){} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java index 1dde85b3c9f10..ab69ab718ff60 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java @@ -117,7 +117,8 @@ else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) { this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; for (int i = 0; i < recordDeserializers.length; i++) { - recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer>(); + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<>( + ioManager.getSpillingDirectoriesPaths()); } watermarks = new long[inputGate.getNumberOfInputChannels()]; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java index 07ada23502ccf..733e7fb376554 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java @@ -139,7 +139,8 @@ else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) { this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()]; for (int i = 0; i < recordDeserializers.length; i++) { - recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer>(); + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<>( + ioManager.getSpillingDirectoriesPaths()); } // determine which unioned channels belong to input 1 and which belong to input 2 diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index c62c881cd3449..b2d019641e824 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.metrics.groups.TaskMetricGroup; @@ -50,6 +49,7 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; + import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -303,7 +303,7 @@ public void acknowledgeCheckpoint(long checkpointId, StateHandle state) { @Override public TaskManagerRuntimeInfo getTaskManagerInfo() { - return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration())); + return new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index deda82fde386c..f054e1865bb4c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -158,7 +158,7 @@ private Task createTask(Class invokable, StreamConf new FiniteDuration(60, TimeUnit.SECONDS), libCache, mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration()), + new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), mock(TaskMetricGroup.class)); } 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 06df46fede9de..5506f55517b38 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 @@ -18,27 +18,28 @@ package org.apache.flink.test.runtime; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.io.network.api.reader.RecordReader; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; -import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.test.util.JavaProgramTestBase; + import org.junit.Ignore; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Arrays; import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.test.util.JavaProgramTestBase; @Ignore public class NetworkStackThroughputITCase { @@ -195,7 +196,11 @@ public static class SpeedTestForwarder extends AbstractInvokable { @Override public void invoke() throws Exception { - RecordReader reader = new RecordReader<>(getEnvironment().getInputGate(0), SpeedTestRecord.class); + RecordReader reader = new RecordReader<>( + getEnvironment().getInputGate(0), + SpeedTestRecord.class, + getEnvironment().getTaskManagerInfo().getTmpDirectories()); + RecordWriter writer = new RecordWriter<>(getEnvironment().getWriter(0)); try { @@ -215,7 +220,10 @@ public static class SpeedTestConsumer extends AbstractInvokable { @Override public void invoke() throws Exception { - RecordReader reader = new RecordReader<>(getEnvironment().getInputGate(0), SpeedTestRecord.class); + RecordReader reader = new RecordReader<>( + getEnvironment().getInputGate(0), + SpeedTestRecord.class, + getEnvironment().getTaskManagerInfo().getTmpDirectories()); try { boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false); From 18015ca9ba45c23cf8f9b8d9664cfd443401ad46 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 25 May 2016 18:39:16 +0200 Subject: [PATCH 67/70] [FLINK-3962] [core] Properly initialize I/O Metric Group --- .../java/org/apache/flink/metrics/groups/TaskMetricGroup.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java index 323853478d179..316c84fd1f866 100644 --- a/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java +++ b/flink-core/src/main/java/org/apache/flink/metrics/groups/TaskMetricGroup.java @@ -65,12 +65,13 @@ protected TaskMetricGroup( this.executionId = executionId; this.subtaskIndex = subtaskIndex; - this.ioMetrics = new IOMetricGroup(registry, this); this.formats.put(SCOPE_TASK_ID, taskId.toString()); this.formats.put(SCOPE_TASK_ATTEMPT, executionId.toString()); this.formats.put(SCOPE_TASK_NAME, checkNotNull(name)); this.formats.put(SCOPE_TASK_SUBTASK_INDEX, String.valueOf(subtaskIndex)); + + this.ioMetrics = new IOMetricGroup(registry, this); } public OperatorMetricGroup addOperator(String name) { From d5ff2269d18d29e61b1386e592f73a0f2eb383ff Mon Sep 17 00:00:00 2001 From: Maximilian Bode Date: Fri, 27 May 2016 15:38:08 +0200 Subject: [PATCH 68/70] Revert to Histogram with internal map --- .../api/common/accumulators/Histogram.java | 44 ++++++------------- 1 file changed, 14 insertions(+), 30 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java index 0c7370baad99e..39131024039e2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java @@ -19,7 +19,6 @@ package org.apache.flink.api.common.accumulators; import org.apache.flink.annotation.Public; -import org.apache.flink.util.MathUtils; import java.util.Map; import java.util.TreeMap; @@ -37,62 +36,47 @@ public class Histogram implements Accumulator private static final long serialVersionUID = 1L; - private LongHistogram internalHistogram = new LongHistogram(); + private TreeMap treeMap = new TreeMap<>(); @Override - public void add(Integer value) { - internalHistogram.add(value); + public void add(final Integer value) { + final Integer current = treeMap.get(value); + final Integer newValue = (current != null ? current : 0) + 1; + this.treeMap.put(value, newValue); } @Override public TreeMap getLocalValue() { - final TreeMap longTreeMap = internalHistogram.getLocalValue(); - return convertToIntMap(longTreeMap); - } - - private TreeMap convertToIntMap(final TreeMap longTreeMap) { - final TreeMap intTreeMap = new TreeMap<>(); - for (final Map.Entry entry : longTreeMap.entrySet()) { - intTreeMap.put(entry.getKey(), checkedCast(entry.getValue())); - } - return intTreeMap; - } - - private int checkedCast(final Long l) { - try { - return MathUtils.checkedDownCast(l); - } catch (final IllegalArgumentException e) { - throw new IllegalArgumentException("Histogram can only deal with int values, consider using LongHistogram.", e); - } + return this.treeMap; } @Override - public void merge(Accumulator> other) { + public void merge(final Accumulator> other) { // Merge the values into this map - for (Map.Entry entryFromOther : other.getLocalValue().entrySet()) { - Long ownValue = internalHistogram.getLocalValue().get(entryFromOther.getKey()); + for (final Map.Entry entryFromOther : other.getLocalValue().entrySet()) { + final Integer ownValue = this.treeMap.get(entryFromOther.getKey()); if (ownValue == null) { - internalHistogram.getLocalValue().put(entryFromOther.getKey(), entryFromOther.getValue().longValue()); + this.treeMap.put(entryFromOther.getKey(), entryFromOther.getValue()); } else { - internalHistogram.getLocalValue().put(entryFromOther.getKey(), entryFromOther.getValue() + ownValue); + this.treeMap.put(entryFromOther.getKey(), entryFromOther.getValue() + ownValue); } } } @Override public void resetLocal() { - internalHistogram.resetLocal(); + this.treeMap.clear(); } @Override public String toString() { - return internalHistogram.toString(); + return this.treeMap.toString(); } @Override public Accumulator> clone() { final Histogram result = new Histogram(); - result.internalHistogram = internalHistogram; + result.treeMap = new TreeMap<>(treeMap); return result; } } From 84fa0dbc514080af3a48d6fc1f2dc8124dd98c12 Mon Sep 17 00:00:00 2001 From: Maximilian Bode Date: Fri, 27 May 2016 15:45:23 +0200 Subject: [PATCH 69/70] [FLINK-3836] Remove HistogramTest. Only tested behavior on int overflow. --- .../common/accumulators/HistogramTest.java | 37 ------------------- 1 file changed, 37 deletions(-) delete mode 100644 flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java diff --git a/flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java b/flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java deleted file mode 100644 index f18cc2332e51a..0000000000000 --- a/flink-core/src/test/java/org/apache/flink/api/common/accumulators/HistogramTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.common.accumulators; - -import org.junit.Test; - -public class HistogramTest { - - private static final Integer KEY = 1; - public static final long MORE_THAN_AN_INT = (long) Integer.MAX_VALUE + 1; - - @Test(expected = IllegalArgumentException.class) - public void longValueThrowsException() { - Histogram histogram = new Histogram(); - for (long i = 0; i < MORE_THAN_AN_INT; i++) { - histogram.add(KEY); - } - - histogram.getLocalValue(); - } -} From a717f71c22c49f45393ff7e6ad7409ff0ae2623a Mon Sep 17 00:00:00 2001 From: Maximilian Bode Date: Fri, 27 May 2016 15:46:05 +0200 Subject: [PATCH 70/70] [FLINK-3836] Deprecate Histogram and getHistogram. Recommend LongHistogram instead. --- .../flink/api/common/accumulators/Histogram.java | 4 ++++ .../flink/api/common/functions/RuntimeContext.java | 14 ++++++++------ 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java index 39131024039e2..ae997a5720c96 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/Histogram.java @@ -30,7 +30,11 @@ * * This class does not extend to continuous values later, because it makes no * attempt to put the data in bins. + * + * @deprecated Usage of {@link LongHistogram} is encouraged. */ +@Deprecated + @Public public class Histogram implements Accumulator> { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 382107fae0f08..d3a4fc4bd65d2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -18,12 +18,8 @@ package org.apache.flink.api.common.functions; -import java.io.Serializable; -import java.util.List; -import java.util.Map; - -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.DoubleCounter; @@ -42,6 +38,10 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.metrics.MetricGroup; +import java.io.Serializable; +import java.util.List; +import java.util.Map; + /** * A RuntimeContext contains information about the context in which functions are executed. Each parallel instance * of the function will have a context through which it can access static contextual information (such as @@ -156,9 +156,11 @@ public interface RuntimeContext { DoubleCounter getDoubleCounter(String name); /** - * Convenience function to create a counter object for histograms. + * Convenience function to create a counter object for histograms that allow only integer values. + * @deprecated Usage of {@link LongHistogram} is encouraged. */ @PublicEvolving + @Deprecated Histogram getHistogram(String name); /**