From b9a866e3ba534dfdc4fb63a956402b8526390e36 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 23 Oct 2018 19:12:58 +0200 Subject: [PATCH 001/359] [FLINK-10490][tests] OperatorSnapshotUtil should use SavepointV2Serializer Please not that state written with OperatorSnapshotUtil before this commit was written in the V1 format. Now we are using the current V2. This closes #6910. --- .../savepoint/SavepointV2Serializer.java | 23 +++++++++++++------ .../streaming/util/OperatorSnapshotUtil.java | 22 +++++++++--------- 2 files changed, 27 insertions(+), 18 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java index faee588c48d7c..fa8407729d90c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.checkpoint.savepoint; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; @@ -67,7 +69,9 @@ * +--------------+---------------------+---------+------+---------------+ * */ -class SavepointV2Serializer implements SavepointSerializer { +@Internal +@VisibleForTesting +public class SavepointV2Serializer implements SavepointSerializer { /** Random magic number for consistency checks */ private static final int MASTER_STATE_MAGIC_NUMBER = 0xc96b1696; @@ -320,7 +324,8 @@ private static OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) keyedStateStream); } - private static void serializeKeyedStateHandle( + @VisibleForTesting + public static void serializeKeyedStateHandle( KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle == null) { @@ -380,7 +385,8 @@ private static Map deserializeStreamStateHandl return result; } - private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { + @VisibleForTesting + public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { final int type = dis.readByte(); if (NULL_HANDLE == type) { @@ -433,7 +439,8 @@ private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) } } - private static void serializeOperatorStateHandle( + @VisibleForTesting + public static void serializeOperatorStateHandle( OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle != null) { @@ -461,7 +468,8 @@ private static void serializeOperatorStateHandle( } } - private static OperatorStateHandle deserializeOperatorStateHandle( + @VisibleForTesting + public static OperatorStateHandle deserializeOperatorStateHandle( DataInputStream dis) throws IOException { final int type = dis.readByte(); @@ -492,7 +500,8 @@ private static OperatorStateHandle deserializeOperatorStateHandle( } } - private static void serializeStreamStateHandle( + @VisibleForTesting + public static void serializeStreamStateHandle( StreamStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle == null) { @@ -518,7 +527,7 @@ private static void serializeStreamStateHandle( dos.flush(); } - private static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { + public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { final int type = dis.read(); if (NULL_HANDLE == type) { return null; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java index 1b5113da2cbbf..53627d5ad4936 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateObjectCollection; -import org.apache.flink.runtime.checkpoint.savepoint.SavepointV1Serializer; +import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2Serializer; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; @@ -55,13 +55,13 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr dos.writeInt(0); // still required for compatibility - SavepointV1Serializer.serializeStreamStateHandle(null, dos); + SavepointV2Serializer.serializeStreamStateHandle(null, dos); Collection rawOperatorState = state.getRawOperatorState(); if (rawOperatorState != null) { dos.writeInt(rawOperatorState.size()); for (OperatorStateHandle operatorStateHandle : rawOperatorState) { - SavepointV1Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); + SavepointV2Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); } } else { // this means no states, not even an empty list @@ -72,7 +72,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr if (managedOperatorState != null) { dos.writeInt(managedOperatorState.size()); for (OperatorStateHandle operatorStateHandle : managedOperatorState) { - SavepointV1Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); + SavepointV2Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); } } else { // this means no states, not even an empty list @@ -83,7 +83,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr if (rawKeyedState != null) { dos.writeInt(rawKeyedState.size()); for (KeyedStateHandle keyedStateHandle : rawKeyedState) { - SavepointV1Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); + SavepointV2Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); } } else { // this means no operator states, not even an empty list @@ -94,7 +94,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr if (managedKeyedState != null) { dos.writeInt(managedKeyedState.size()); for (KeyedStateHandle keyedStateHandle : managedKeyedState) { - SavepointV1Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); + SavepointV2Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); } } else { // this means no operator states, not even an empty list @@ -113,14 +113,14 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti dis.readInt(); // still required for compatibility to consume the bytes. - SavepointV1Serializer.deserializeStreamStateHandle(dis); + SavepointV2Serializer.deserializeStreamStateHandle(dis); List rawOperatorState = null; int numRawOperatorStates = dis.readInt(); if (numRawOperatorStates >= 0) { rawOperatorState = new ArrayList<>(); for (int i = 0; i < numRawOperatorStates; i++) { - OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle( + OperatorStateHandle operatorState = SavepointV2Serializer.deserializeOperatorStateHandle( dis); rawOperatorState.add(operatorState); } @@ -131,7 +131,7 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti if (numManagedOperatorStates >= 0) { managedOperatorState = new ArrayList<>(); for (int i = 0; i < numManagedOperatorStates; i++) { - OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle( + OperatorStateHandle operatorState = SavepointV2Serializer.deserializeOperatorStateHandle( dis); managedOperatorState.add(operatorState); } @@ -142,7 +142,7 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti if (numRawKeyedStates >= 0) { rawKeyedState = new ArrayList<>(); for (int i = 0; i < numRawKeyedStates; i++) { - KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle( + KeyedStateHandle keyedState = SavepointV2Serializer.deserializeKeyedStateHandle( dis); rawKeyedState.add(keyedState); } @@ -153,7 +153,7 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti if (numManagedKeyedStates >= 0) { managedKeyedState = new ArrayList<>(); for (int i = 0; i < numManagedKeyedStates; i++) { - KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle( + KeyedStateHandle keyedState = SavepointV2Serializer.deserializeKeyedStateHandle( dis); managedKeyedState.add(keyedState); } From 08cd6ea7cd8afa8d2761dde521eb9a7bf21ec5e6 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 5 Nov 2018 11:07:22 +0100 Subject: [PATCH 002/359] [FLINK-10368][e2e] Hardened kerberized yarn e2e test * wait for whole bootstrapping script to execute on master node before submitting job * retrying to start hadoop cluster. Failling test in case could not start hadoop cluster. * added check that all containers are up and running before submitting job * reduced memory requirements for the kerberized yarn test --- .../docker-hadoop-secure-cluster/bootstrap.sh | 1 + .../config/yarn-site.xml | 9 +- .../test-scripts/test_yarn_kerberos_docker.sh | 139 ++++++++++++------ 3 files changed, 100 insertions(+), 49 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh index 7b5e50ba4390a..5b98b96e51da1 100755 --- a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh +++ b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh @@ -124,6 +124,7 @@ elif [ "$1" == "master" ]; then hdfs dfs -chown hadoop-user:hadoop-user /user/hadoop-user kdestroy + echo "Finished master initialization" while true; do sleep 1000; done elif [ "$1" == "worker" ]; then diff --git a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml index 9b17acc1656ca..c7736a694899b 100644 --- a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml +++ b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml @@ -21,6 +21,11 @@ under the License. mapreduce_shuffle + + yarn.nodemanager.vmem-pmem-ratio + 3 + + @@ -33,12 +38,12 @@ under the License. are scheduled on one NM, which wouldn't provoke a previously fixed Kerberos keytab bug. --> yarn.nodemanager.resource.memory-mb - 4100 + 2500 yarn.scheduler.minimum-allocation-mb - 2000 + 1000 diff --git a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh index c9ef15d3dc533..5f2dea2ea6a01 100755 --- a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh +++ b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh @@ -24,7 +24,8 @@ FLINK_TARBALL_DIR=$TEST_DATA_DIR FLINK_TARBALL=flink.tar.gz FLINK_DIRNAME=$(basename $FLINK_DIR) -MAX_RETRY_SECONDS=800 +MAX_RETRY_SECONDS=120 +CLUSTER_SETUP_RETRIES=3 echo "Flink Tarball directory $FLINK_TARBALL_DIR" echo "Flink tarball filename $FLINK_TARBALL" @@ -33,20 +34,6 @@ echo "End-to-end directory $END_TO_END_DIR" docker --version docker-compose --version -mkdir -p $FLINK_TARBALL_DIR -tar czf $FLINK_TARBALL_DIR/$FLINK_TARBALL -C $(dirname $FLINK_DIR) . - -echo "Building Hadoop Docker container" -until docker build --build-arg HADOOP_VERSION=2.8.4 -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/Dockerfile -t flink/docker-hadoop-secure-cluster:latest $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/; do - # with all the downloading and ubuntu updating a lot of flakiness can happen, make sure - # we don't immediately fail - echo "Something went wrong while building the Docker image, retrying ..." - sleep 2 -done - -echo "Starting Hadoop cluster" -docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml up -d - # make sure we stop our cluster at the end function cluster_shutdown { # don't call ourselves again for another signal interruption @@ -60,12 +47,71 @@ function cluster_shutdown { trap cluster_shutdown INT trap cluster_shutdown EXIT -until docker cp $FLINK_TARBALL_DIR/$FLINK_TARBALL master:/home/hadoop-user/; do - # we're retrying this one because we don't know yet if the container is ready - echo "Uploading Flink tarball to docker master failed, retrying ..." - sleep 5 +function start_hadoop_cluster() { + echo "Starting Hadoop cluster" + docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml up -d + + # wait for kerberos to be set up + start_time=$(date +%s) + until docker logs master 2>&1 | grep -q "Finished master initialization"; do + current_time=$(date +%s) + time_diff=$((current_time - start_time)) + + if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then + return 1 + else + echo "Waiting for hadoop cluster to come up. We have been trying for $time_diff seconds, retrying ..." + sleep 10 + fi + done + + # perform health checks + if ! { [ $(docker inspect -f '{{.State.Running}}' master 2>&1) = 'true' ] && + [ $(docker inspect -f '{{.State.Running}}' slave1 2>&1) = 'true' ] && + [ $(docker inspect -f '{{.State.Running}}' slave2 2>&1) = 'true' ] && + [ $(docker inspect -f '{{.State.Running}}' kdc 2>&1) = 'true' ]; }; + then + return 1 + fi + + return 0 +} + +mkdir -p $FLINK_TARBALL_DIR +tar czf $FLINK_TARBALL_DIR/$FLINK_TARBALL -C $(dirname $FLINK_DIR) . + +echo "Building Hadoop Docker container" +until docker build --build-arg HADOOP_VERSION=2.8.4 \ + -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/Dockerfile \ + -t flink/docker-hadoop-secure-cluster:latest \ + $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/; +do + # with all the downloading and ubuntu updating a lot of flakiness can happen, make sure + # we don't immediately fail + echo "Something went wrong while building the Docker image, retrying ..." + sleep 2 +done + +CLUSTER_STARTED=1 +for (( i = 0; i < $CLUSTER_SETUP_RETRIES; i++ )) +do + if start_hadoop_cluster; then + echo "Cluster started successfully." + CLUSTER_STARTED=0 + break #continue test, cluster set up succeeded + fi + + echo "ERROR: Could not start hadoop cluster. Retrying..." + docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml down done +if [[ ${CLUSTER_STARTED} -ne 0 ]]; then + echo "ERROR: Could not start hadoop cluster. Aborting..." + exit 1 +fi + +docker cp $FLINK_TARBALL_DIR/$FLINK_TARBALL master:/home/hadoop-user/ + # now, at least the container is ready docker exec -it master bash -c "tar xzf /home/hadoop-user/$FLINK_TARBALL --directory /home/hadoop-user/" @@ -73,6 +119,7 @@ docker exec -it master bash -c "tar xzf /home/hadoop-user/$FLINK_TARBALL --direc docker exec -it master bash -c "echo \"security.kerberos.login.keytab: /home/hadoop-user/hadoop-user.keytab\" > /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" docker exec -it master bash -c "echo \"security.kerberos.login.principal: hadoop-user\" >> /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" docker exec -it master bash -c "echo \"slot.request.timeout: 60000\" >> /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" +docker exec -it master bash -c "echo \"containerized.heap-cutoff-min: 100\" >> /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" echo "Flink config:" docker exec -it master bash -c "cat /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" @@ -84,33 +131,28 @@ OUTPUT_PATH=hdfs:///user/hadoop-user/wc-out-$RANDOM start_time=$(date +%s) # it's important to run this with higher parallelism, otherwise we might risk that # JM and TM are on the same YARN node and that we therefore don't test the keytab shipping -until docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -yn 3 -ys 1 -ytm 2000 -yjm 2000 -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH"; do - current_time=$(date +%s) - time_diff=$((current_time - start_time)) - - if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then - echo "We tried running the job for $time_diff seconds, max is $MAX_RETRY_SECONDS seconds, aborting" - mkdir -p $TEST_DATA_DIR/logs - echo "Hadoop logs:" - docker cp master:/var/log/hadoop/* $TEST_DATA_DIR/logs/ - for f in $TEST_DATA_DIR/logs/*; do - echo "$f:" - cat $f - done - echo "Docker logs:" - docker logs master - exit 1 - else - echo "Running the Flink job failed, might be that the cluster is not ready yet. We have been trying for $time_diff seconds, retrying ..." - sleep 5 - fi -done - -docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" -docker exec -it master bash -c "hdfs dfs -ls $OUTPUT_PATH" -OUTPUT=$(docker exec -it master bash -c "hdfs dfs -cat $OUTPUT_PATH/*") -docker exec -it master bash -c "kdestroy" -echo "$OUTPUT" +if docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ + /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -yn 3 -ys 1 -ytm 1000 -yjm 1000 \ + -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH"; +then + docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" + docker exec -it master bash -c "hdfs dfs -ls $OUTPUT_PATH" + OUTPUT=$(docker exec -it master bash -c "hdfs dfs -cat $OUTPUT_PATH/*") + docker exec -it master bash -c "kdestroy" + echo "$OUTPUT" +else + echo "Running the job failed." + mkdir -p $TEST_DATA_DIR/logs + echo "Hadoop logs:" + docker cp master:/var/log/hadoop/* $TEST_DATA_DIR/logs/ + for f in $TEST_DATA_DIR/logs/*; do + echo "$f:" + cat $f + done + echo "Docker logs:" + docker logs master + exit 1 +fi if [[ ! "$OUTPUT" =~ "consummation,1" ]]; then echo "Output does not contain (consummation, 1) as required" @@ -139,7 +181,10 @@ fi echo "Running Job without configured keytab, the exception you see below is expected" docker exec -it master bash -c "echo \"\" > /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" # verify that it doesn't work if we don't configure a keytab -OUTPUT=$(docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -yn 3 -ys 1 -ytm 1200 -yjm 800 -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH") +OUTPUT=$(docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ + /home/hadoop-user/$FLINK_DIRNAME/bin/flink run \ + -m yarn-cluster -yn 3 -ys 1 -ytm 1000 -yjm 1000 -p 3 \ + /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH") echo "$OUTPUT" if [[ ! "$OUTPUT" =~ "Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials" ]]; then From 5c28c2c1f89535979ffc881eb4a3fd48fa0c6fa9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 4 Nov 2018 18:38:17 +0100 Subject: [PATCH 003/359] [FLINK-10772][release] Fix create_binary_release.sh Remove the unnecessary call to change_scala_version.sh and remove the -Dmaven.test.skip=true property. The latter is necessary because this property suppresses the compilation and packaging of test classes. It, however, does not suppress the resolution of test dependencies which will then fail to compile because test dependencies have not been built. This commit also removes the redundant call to build flink-shaded/hadoop/flink-shaded-hadoop2-uber which is a dependency of flink-dist anyway. --- tools/releasing/create_binary_release.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index 814612791cbb3..f4d7c5770ca9c 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -78,8 +78,7 @@ make_binary_release() { fi # enable release profile here (to check for the maven version) - tools/change-scala-version.sh ${SCALA_VERSION} - $MVN clean package $FLAGS -Prelease -pl flink-shaded-hadoop/flink-shaded-hadoop2-uber,flink-dist -am -Dgpg.skip -Dcheckstyle.skip=true -DskipTests -Dmaven.test.skip=true + $MVN clean package $FLAGS -Prelease -pl flink-dist -am -Dgpg.skip -Dcheckstyle.skip=true -DskipTests cd flink-dist/target/flink-*-bin/ tar czf "${dir_name}.tgz" flink-* From 80bed3d87b2722f0f6c04930b3ab1bdde860962c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 4 Nov 2018 19:32:44 +0100 Subject: [PATCH 004/359] [FLINK-10773] Harden resume externalized checkpoint end-to-end test Ignore the 'Artificial Failure' exceptions and rename ExceptionThrowingFailureMapper into FailureMapper to avoid false positive exception matchings. --- .../streaming/tests/DataStreamAllroundTestJobFactory.java | 6 ++++-- .../streaming/tests/DataStreamAllroundTestProgram.java | 6 +++--- ...ceptionThrowingFailureMapper.java => FailureMapper.java} | 4 ++-- flink-end-to-end-tests/test-scripts/common.sh | 1 + 4 files changed, 10 insertions(+), 7 deletions(-) rename flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/{ExceptionThrowingFailureMapper.java => FailureMapper.java} (94%) diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java index fb92960bb8684..3c8d0ad537faa 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java @@ -322,6 +322,8 @@ static BoundedOutOfOrdernessTimestampExtractor createTimestampExtractor(P SEQUENCE_GENERATOR_SRC_EVENT_TIME_MAX_OUT_OF_ORDERNESS.key(), SEQUENCE_GENERATOR_SRC_EVENT_TIME_MAX_OUT_OF_ORDERNESS.defaultValue()))) { + private static final long serialVersionUID = -3154419724891779938L; + @Override public long extractTimestamp(Event element) { return element.getEventTime(); @@ -367,8 +369,8 @@ static boolean isSimulateFailures(ParameterTool pt) { return pt.getBoolean(TEST_SIMULATE_FAILURE.key(), TEST_SIMULATE_FAILURE.defaultValue()); } - static MapFunction createExceptionThrowingFailureMapper(ParameterTool pt) { - return new ExceptionThrowingFailureMapper<>( + static MapFunction createFailureMapper(ParameterTool pt) { + return new FailureMapper<>( pt.getLong( TEST_SIMULATE_FAILURE_NUM_RECORDS.key(), TEST_SIMULATE_FAILURE_NUM_RECORDS.defaultValue()), diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java index 70fdade4a1c67..b14e2af1b52bd 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java @@ -41,7 +41,7 @@ import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createArtificialKeyedStateMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createArtificialOperatorStateMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createEventSource; -import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createExceptionThrowingFailureMapper; +import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createFailureMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createSemanticsCheckMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createTimestampExtractor; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.isSimulateFailures; @@ -68,7 +68,7 @@ public class DataStreamAllroundTestProgram { private static final String OPERATOR_STATE_OPER_NAME = "ArtificalOperatorStateMapper"; private static final String TIME_WINDOW_OPER_NAME = "TumblingWindowOperator"; private static final String SEMANTICS_CHECK_MAPPER_NAME = "SemanticsCheckMapper"; - private static final String FAILURE_MAPPER_NAME = "ExceptionThrowingFailureMapper"; + private static final String FAILURE_MAPPER_NAME = "FailureMapper"; public static void main(String[] args) throws Exception { final ParameterTool pt = ParameterTool.fromArgs(args); @@ -145,7 +145,7 @@ public void apply(Integer integer, TimeWindow window, Iterable input, Col if (isSimulateFailures(pt)) { eventStream3 = eventStream3 - .map(createExceptionThrowingFailureMapper(pt)) + .map(createFailureMapper(pt)) .setParallelism(1) .name(FAILURE_MAPPER_NAME); } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/ExceptionThrowingFailureMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/FailureMapper.java similarity index 94% rename from flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/ExceptionThrowingFailureMapper.java rename to flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/FailureMapper.java index d758ef5cf0d1b..a3a1c253fc070 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/ExceptionThrowingFailureMapper.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/FailureMapper.java @@ -30,7 +30,7 @@ * of the operator can also be configured. Note that this also takes into account * failures that were not triggered by this mapper, e.g. TaskManager failures. */ -public class ExceptionThrowingFailureMapper extends RichMapFunction implements CheckpointListener { +public class FailureMapper extends RichMapFunction implements CheckpointListener { private static final long serialVersionUID = -5286927943454740016L; @@ -41,7 +41,7 @@ public class ExceptionThrowingFailureMapper extends RichMapFunction imp private long numProcessedRecords; private long numCompleteCheckpoints; - public ExceptionThrowingFailureMapper( + public FailureMapper( long numProcessedRecordsFailureThreshold, long numCompleteCheckpointsFailureThreshold, int maxNumFailures) { diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 09c42af8742ea..bdf6f64c5a8f2 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -347,6 +347,7 @@ function check_logs_for_exceptions { | grep -v "java.lang.Exception: Execution was suspended" \ | grep -v "java.io.InvalidClassException: org.apache.flink.formats.avro.typeutils.AvroSerializer" \ | grep -v "Caused by: java.lang.Exception: JobManager is shutting down" \ + | grep -v "java.lang.Exception: Artificial failure" \ | grep -ic "exception") if [[ ${exception_count} -gt 0 ]]; then echo "Found exception in log files:" From 4556c8f4d1ddf6d8fe19fd6fae061e879a981ca5 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 31 Oct 2018 11:57:56 +0100 Subject: [PATCH 005/359] [FLINK-10364][tests] Fix instability in NonHAQueryableStateFsBackendITCase#testMapState This closes #6975. --- .../itcases/AbstractQueryableStateTestBase.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index e99a28b36a212..83976f16094a4 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -836,9 +836,11 @@ public void processElement(Tuple2 value, Context ctx, Collector value = future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get(key); - assertEquals("Key mismatch", key, value.f0.intValue()); - if (expected == value.f1) { + Tuple2 value = + future.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).get(key); + + if (value != null && value.f0 != null && expected == value.f1) { + assertEquals("Key mismatch", key, value.f0.intValue()); success = true; } else { // Retry From 97ac1ab0cc513511a0b61b52663e451c0e52218b Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 1 Nov 2018 12:19:14 +0100 Subject: [PATCH 006/359] [FLINK-10638][table] Invalid table scan resolution for temporal join queries Previously there was a strict fixed order of applying LogicalCorrelateToTemporalTableJoinRule and TableScanRule rules. This was causing problems, since either of them could create a new RelNodes that have to be subject of the other rule (imagine deeply nested TemporalTableFunction that references registered tables/views and other TemporalTableFunctions). Solution to this problem is to run both of those rules in one group/collection in HepPlaner. Instead of applying one rule to whole tree then the other rule, both rules are applied to a parent node, before going down/deeper. --- .../table/api/BatchTableEnvironment.scala | 5 +- .../table/api/StreamTableEnvironment.scala | 8 +-- .../flink/table/api/TableEnvironment.scala | 64 +++++++++++++------ .../table/plan/rules/FlinkRuleSets.scala | 16 ++--- .../stream/sql/TemporalTableJoinTest.scala | 4 +- .../stream/table/TemporalTableJoinTest.scala | 7 +- .../stream/sql/TemporalJoinITCase.scala | 10 +-- 7 files changed, 72 insertions(+), 42 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index 6a7a921bffbe3..99e9d7e6d011b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -449,9 +449,8 @@ abstract class BatchTableEnvironment( */ private[flink] def optimize(relNode: RelNode): RelNode = { val convSubQueryPlan = optimizeConvertSubQueries(relNode) - val temporalTableJoinPlan = optimizeConvertToTemporalJoin(convSubQueryPlan) - val fullNode = optimizeConvertTableReferences(temporalTableJoinPlan) - val decorPlan = RelDecorrelator.decorrelateQuery(fullNode) + val expandedPlan = optimizeExpandPlan(convSubQueryPlan) + val decorPlan = RelDecorrelator.decorrelateQuery(expandedPlan) val normalizedPlan = optimizeNormalizeLogicalPlan(decorPlan) val logicalPlan = optimizeLogicalPlan(normalizedPlan) optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASET) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index 4973f34147f61..8c6a1e0a04bf4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -804,13 +804,13 @@ abstract class StreamTableEnvironment( */ private[flink] def optimize(relNode: RelNode, updatesAsRetraction: Boolean): RelNode = { val convSubQueryPlan = optimizeConvertSubQueries(relNode) - val temporalTableJoinPlan = optimizeConvertToTemporalJoin(convSubQueryPlan) - val fullNode = optimizeConvertTableReferences(temporalTableJoinPlan) - val decorPlan = RelDecorrelator.decorrelateQuery(fullNode) + val expandedPlan = optimizeExpandPlan(convSubQueryPlan) + val decorPlan = RelDecorrelator.decorrelateQuery(expandedPlan) val planWithMaterializedTimeAttributes = RelTimeIndicatorConverter.convert(decorPlan, getRelBuilder.getRexBuilder) val normalizedPlan = optimizeNormalizeLogicalPlan(planWithMaterializedTimeAttributes) val logicalPlan = optimizeLogicalPlan(normalizedPlan) + val physicalPlan = optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASTREAM) optimizeDecoratePlan(physicalPlan, updatesAsRetraction) } @@ -827,7 +827,7 @@ abstract class StreamTableEnvironment( } else { relNode } - runHepPlanner( + runHepPlannerSequentially( HepMatchOrder.BOTTOM_UP, decoRuleSet, planToDecorate, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index 58831d1027122..26f9e50fedd01 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableList import org.apache.calcite.config.Lex import org.apache.calcite.jdbc.CalciteSchema import org.apache.calcite.plan.RelOptPlanner.CannotPlanException -import org.apache.calcite.plan.hep.{HepMatchOrder, HepPlanner, HepProgramBuilder} +import org.apache.calcite.plan.hep.{HepMatchOrder, HepPlanner, HepProgram, HepProgramBuilder} import org.apache.calcite.plan.{Convention, RelOptPlanner, RelOptUtil, RelTraitSet} import org.apache.calcite.rel.RelNode import org.apache.calcite.schema.SchemaPlus @@ -256,34 +256,31 @@ abstract class TableEnvironment(val config: TableConfig) { protected def getBuiltInPhysicalOptRuleSet: RuleSet protected def optimizeConvertSubQueries(relNode: RelNode): RelNode = { - runHepPlanner( + runHepPlannerSequentially( HepMatchOrder.BOTTOM_UP, FlinkRuleSets.TABLE_SUBQUERY_RULES, relNode, relNode.getTraitSet) } - protected def optimizeConvertToTemporalJoin(relNode: RelNode): RelNode = { - runHepPlanner( - HepMatchOrder.BOTTOM_UP, - FlinkRuleSets.TEMPORAL_JOIN_RULES, + protected def optimizeExpandPlan(relNode: RelNode): RelNode = { + val result = runHepPlannerSimultaneously( + HepMatchOrder.TOP_DOWN, + FlinkRuleSets.EXPAND_PLAN_RULES, relNode, relNode.getTraitSet) - } - protected def optimizeConvertTableReferences(relNode: RelNode): RelNode = { - runHepPlanner( - HepMatchOrder.BOTTOM_UP, - FlinkRuleSets.TABLE_REF_RULES, - relNode, - relNode.getTraitSet) + runHepPlannerSequentially( + HepMatchOrder.TOP_DOWN, + FlinkRuleSets.POST_EXPAND_CLEAN_UP_RULES, + result, + result.getTraitSet) } - protected def optimizeNormalizeLogicalPlan(relNode: RelNode): RelNode = { val normRuleSet = getNormRuleSet if (normRuleSet.iterator().hasNext) { - runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, relNode, relNode.getTraitSet) + runHepPlannerSequentially(HepMatchOrder.BOTTOM_UP, normRuleSet, relNode, relNode.getTraitSet) } else { relNode } @@ -310,13 +307,16 @@ abstract class TableEnvironment(val config: TableConfig) { } /** - * run HEP planner + * run HEP planner with rules applied one by one. First apply one rule to all of the nodes + * and only then apply the next rule. If a rule creates a new node preceding rules will not + * be applied to the newly created node. */ - protected def runHepPlanner( + protected def runHepPlannerSequentially( hepMatchOrder: HepMatchOrder, ruleSet: RuleSet, input: RelNode, targetTraits: RelTraitSet): RelNode = { + val builder = new HepProgramBuilder builder.addMatchOrder(hepMatchOrder) @@ -324,8 +324,36 @@ abstract class TableEnvironment(val config: TableConfig) { while (it.hasNext) { builder.addRuleInstance(it.next()) } + runHepPlanner(builder.build(), input, targetTraits) + } + + /** + * run HEP planner with rules applied simultaneously. Apply all of the rules to the given + * node before going to the next one. If a rule creates a new node all of the rules will + * be applied to this new node. + */ + protected def runHepPlannerSimultaneously( + hepMatchOrder: HepMatchOrder, + ruleSet: RuleSet, + input: RelNode, + targetTraits: RelTraitSet): RelNode = { + + val builder = new HepProgramBuilder + builder.addMatchOrder(hepMatchOrder) + + builder.addRuleCollection(ruleSet.asScala.toList.asJava) + runHepPlanner(builder.build(), input, targetTraits) + } + + /** + * run HEP planner + */ + protected def runHepPlanner( + hepProgram: HepProgram, + input: RelNode, + targetTraits: RelTraitSet): RelNode = { - val planner = new HepPlanner(builder.build, frameworkConfig.getContext) + val planner = new HepPlanner(hepProgram, frameworkConfig.getContext) planner.setRoot(input) if (input.getTraitSet != targetTraits) { planner.changeTraits(input, targetTraits.simplify) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 6e2ccdeba5b04..5e0ee32ad6c5d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -39,18 +39,14 @@ object FlinkRuleSets { SubQueryRemoveRule.JOIN) /** - * Handles proper conversion of correlate queries with temporal table functions - * into temporal table joins. This can create new table scans in the plan. + * Expand plan by replacing references to tables into a proper plan sub trees. Those rules + * can create new plan nodes. */ - val TEMPORAL_JOIN_RULES: RuleSet = RuleSets.ofList( - LogicalCorrelateToTemporalTableJoinRule.INSTANCE - ) + val EXPAND_PLAN_RULES: RuleSet = RuleSets.ofList( + LogicalCorrelateToTemporalTableJoinRule.INSTANCE, + TableScanRule.INSTANCE) - /** - * Convert table references before query decorrelation. - */ - val TABLE_REF_RULES: RuleSet = RuleSets.ofList( - TableScanRule.INSTANCE, + val POST_EXPAND_CLEAN_UP_RULES: RuleSet = RuleSets.ofList( EnumerableToLogicalTableScan.INSTANCE) val LOGICAL_OPT_RULES: RuleSet = RuleSets.ofList( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala index 3c47f562aae47..27c40bbbef24e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala @@ -87,7 +87,9 @@ class TemporalTableJoinTest extends TableTestBase { val ratesHistory = util.addTable[(Timestamp, String, String, Int, Int)]( "RatesHistory", 'rowtime.rowtime, 'comment, 'currency, 'rate, 'secondary_key) - val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency) + val rates = ratesHistory + .filter('rate > 110L) + .createTemporalTableFunction('rowtime, 'currency) util.addFunction("Rates", rates) val sqlQuery = diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala index f8d49238430e7..299c14417b62d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala @@ -87,7 +87,9 @@ class TemporalTableJoinTest extends TableTestBase { val ratesHistory = util.addTable[(Timestamp, String, String, Int, Int)]( "RatesHistory", 'rowtime.rowtime, 'comment, 'currency, 'rate, 'secondary_key) - val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency) + val rates = ratesHistory + .filter('rate > 110L) + .createTemporalTableFunction('rowtime, 'currency) util.addFunction("Rates", rates) val result = orders @@ -226,7 +228,8 @@ object TemporalTableJoinTest { unaryNode( "DataStreamCalc", streamTableNode(2), - term("select", "rowtime, currency, rate, secondary_key") + term("select", "rowtime, currency, rate, secondary_key"), + term("where", ">(rate, 110)") ), term("where", "AND(" + diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala index df0f01be0d58a..0fb175370fbfe 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala @@ -127,8 +127,6 @@ class TemporalJoinITCase extends StreamingWithStateTestBase { var expectedOutput = new mutable.HashSet[String]() expectedOutput += (2 * 114).toString - expectedOutput += (1 * 102).toString - expectedOutput += (50 * 1).toString expectedOutput += (3 * 116).toString val orders = env @@ -142,11 +140,15 @@ class TemporalJoinITCase extends StreamingWithStateTestBase { tEnv.registerTable("Orders", orders) tEnv.registerTable("RatesHistory", ratesHistory) + tEnv.registerTable("FilteredRatesHistory", tEnv.scan("RatesHistory").filter('rate > 110L)) tEnv.registerFunction( "Rates", - ratesHistory.createTemporalTableFunction('rowtime, 'currency)) + tEnv.scan("FilteredRatesHistory").createTemporalTableFunction('rowtime, 'currency)) + tEnv.registerTable("TemporalJoinResult", tEnv.sqlQuery(sqlQuery)) - val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + // Scan from registered table to test for interplay between + // LogicalCorrelateToTemporalTableJoinRule and TableScanRule + val result = tEnv.scan("TemporalJoinResult").toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() From 6128ff88fd47096c2056a90f07d1fd8eb486344d Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 25 Oct 2018 17:11:49 +0200 Subject: [PATCH 007/359] [FLINK-10627][e2e] Test s3 output for streaming file sink. --- .../flink-e2e-test-utils/pom.xml | 71 ++++++ .../streaming/tests/util/s3/S3QueryUtil.java | 92 +++++++ .../tests/util/s3/S3UtilProgram.java | 225 ++++++++++++++++ flink-end-to-end-tests/pom.xml | 1 + flink-end-to-end-tests/run-nightly-tests.sh | 1 + flink-end-to-end-tests/test-scripts/common.sh | 54 ++-- .../test-scripts/common_s3.sh | 240 ++++++++++++++++++ .../test-scripts/test_shaded_hadoop_s3a.sh | 23 +- .../test-scripts/test_shaded_presto_s3.sh | 21 +- .../test-scripts/test_streaming_file_sink.sh | 80 ++++-- 10 files changed, 710 insertions(+), 98 deletions(-) create mode 100644 flink-end-to-end-tests/flink-e2e-test-utils/pom.xml create mode 100644 flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3QueryUtil.java create mode 100644 flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3UtilProgram.java create mode 100644 flink-end-to-end-tests/test-scripts/common_s3.sh diff --git a/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml b/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml new file mode 100644 index 0000000000000..219d6626ef373 --- /dev/null +++ b/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml @@ -0,0 +1,71 @@ + + + + + flink-end-to-end-tests + org.apache.flink + 1.7-SNAPSHOT + + 4.0.0 + + flink-e2e-test-utils + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + com.amazonaws + aws-java-sdk-s3 + 1.11.437 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + S3UtilProgram + package + + shade + + + S3UtilProgram + + + org.apache.flink.streaming.tests.util.s3.S3UtilProgram + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3QueryUtil.java b/flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3QueryUtil.java new file mode 100644 index 0000000000000..781a8edd20aca --- /dev/null +++ b/flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3QueryUtil.java @@ -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.streaming.tests.util.s3; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.CSVInput; +import com.amazonaws.services.s3.model.CSVOutput; +import com.amazonaws.services.s3.model.CompressionType; +import com.amazonaws.services.s3.model.ExpressionType; +import com.amazonaws.services.s3.model.InputSerialization; +import com.amazonaws.services.s3.model.OutputSerialization; +import com.amazonaws.services.s3.model.SelectObjectContentEvent; +import com.amazonaws.services.s3.model.SelectObjectContentEventStream; +import com.amazonaws.services.s3.model.SelectObjectContentEventVisitor; +import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.SelectObjectContentResult; + +import java.io.ByteArrayOutputStream; +import java.io.InputStream; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.amazonaws.util.IOUtils.copy; + +class S3QueryUtil { + /** Run SQL query over non-compressed CSV file saved in s3 object. */ + static String queryFile( + AmazonS3 s3client, String bucket, String s3file, @SuppressWarnings("SameParameterValue") String query) { + SelectObjectContentRequest request = generateBaseCSVRequest(bucket, s3file, query); + final AtomicBoolean isResultComplete = new AtomicBoolean(false); + String res; + try (SelectObjectContentResult result = s3client.selectObjectContent(request); + SelectObjectContentEventStream payload = result.getPayload(); + ByteArrayOutputStream out = new ByteArrayOutputStream()) { + InputStream resultInputStream = payload.getRecordsInputStream( + new SelectObjectContentEventVisitor() { + @Override + public void visit(SelectObjectContentEvent.EndEvent event) { + isResultComplete.set(true); + } + } + ); + copy(resultInputStream, out); + res = out.toString().trim(); + } catch (Throwable e) { + System.out.println("SQL query failure"); + throw new RuntimeException("SQL query failure", e); + } + /* + * The End Event indicates all matching records have been transmitted. + * If the End Event is not received, the results may be incomplete. + */ + if (!isResultComplete.get()) { + throw new RuntimeException("S3 Select request was incomplete as End Event was not received."); + } + return res; + } + + private static SelectObjectContentRequest generateBaseCSVRequest(String bucket, String key, String query) { + SelectObjectContentRequest request = new SelectObjectContentRequest(); + request.setBucketName(bucket); + request.setKey(key); + request.setExpression(query); + request.setExpressionType(ExpressionType.SQL); + + InputSerialization inputSerialization = new InputSerialization(); + inputSerialization.setCsv(new CSVInput()); + inputSerialization.setCompressionType(CompressionType.NONE); + request.setInputSerialization(inputSerialization); + + OutputSerialization outputSerialization = new OutputSerialization(); + outputSerialization.setCsv(new CSVOutput()); + request.setOutputSerialization(outputSerialization); + + return request; + } +} diff --git a/flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3UtilProgram.java b/flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3UtilProgram.java new file mode 100644 index 0000000000000..70cb55854c573 --- /dev/null +++ b/flink-end-to-end-tests/flink-e2e-test-utils/src/main/java/org/apache/flink/streaming/tests/util/s3/S3UtilProgram.java @@ -0,0 +1,225 @@ +/* + * 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.tests.util.s3; + +import org.apache.flink.api.java.utils.ParameterTool; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.amazonaws.services.s3.transfer.KeyFilter; +import com.amazonaws.services.s3.transfer.TransferManager; +import com.amazonaws.services.s3.transfer.TransferManagerBuilder; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * S3 utilities. + * + *

Usage: java -jar S3UtilProgram.jar args. + * + *

Note: {@code S3UtilProgram.Action.lineNumber*} actions are applicable only + * to valid non-compressed CSV comma separated files. + * + *

Program parameters: + *

    + *
  • action (string, required): Action to perform, see {@link S3UtilProgram.Action}.
  • + *
  • bucket (string, required): Bucket where s3 objects reside.
  • + *
  • s3file (string, required for single object actions): s3 object key.
  • + *
  • s3prefix (string, required for actions over objects grouped by key prefix): s3 key prefix.
  • + *
  • s3filePrefix (string, optional for downloadByFullPathAndFileNamePrefix or numberOfLinesInFilesWithFullAndNamePrefix): + * s3 file name prefix w/o directory to filter files by name.
  • + *
  • localFile (string, required for single file actions): local file path.
  • + *
  • localFolder (string, required for actions over folders): local folder path.
  • + *
  • parallelism (int, default 10): parallelism for parallelizable actions + * (e.g. {@code numberOfLinesInFilesWithFullAndNamePrefix}).
  • + *
+ */ +class S3UtilProgram { + enum Action { + listByFullPathPrefix, + downloadFile, + downloadByFullPathAndFileNamePrefix, + deleteFile, + deleteByFullPathPrefix, + numberOfLinesInFile, + numberOfLinesInFilesWithFullAndNamePrefix + } + + private static final Map> handlers; + static { + Map> handlersMutable = new HashMap<>(); + handlersMutable.put(Action.listByFullPathPrefix, S3UtilProgram::listByFullPathPrefix); + handlersMutable.put(Action.downloadFile, S3UtilProgram::downloadFile); + handlersMutable.put(Action.downloadByFullPathAndFileNamePrefix, + S3UtilProgram::downloadByFullPathAndFileNamePrefix); + handlersMutable.put(Action.deleteFile, S3UtilProgram::deleteFile); + handlersMutable.put(Action.deleteByFullPathPrefix, S3UtilProgram::deleteByFullPathPrefix); + handlersMutable.put(Action.numberOfLinesInFile, S3UtilProgram::numberOfLinesInFile); + handlersMutable.put(Action.numberOfLinesInFilesWithFullAndNamePrefix, + S3UtilProgram::numberOfLinesInFilesWithFullAndNamePrefix); + handlers = Collections.unmodifiableMap(handlersMutable); + } + + private static final String countQuery = "select count(*) from s3object"; + + public static void main(String[] args) { + final ParameterTool params = ParameterTool.fromArgs(args); + final Action action = Action.valueOf(params.getRequired("action")); + handlers.get(action).accept(params); + } + + private static void listByFullPathPrefix(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3prefix = params.getRequired("s3prefix"); + listByFullPathPrefix(bucket, s3prefix).forEach(System.out::println); + } + + private static List listByFullPathPrefix(final String bucket, final String s3prefix) { + return AmazonS3ClientBuilder.defaultClient().listObjects(bucket, s3prefix).getObjectSummaries() + .stream().map(S3ObjectSummary::getKey).collect(Collectors.toList()); + } + + private static void downloadFile(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3file = params.getRequired("s3file"); + final String localFile = params.getRequired("localFile"); + TransferManager tx = TransferManagerBuilder.defaultTransferManager(); + try { + tx.download(bucket, s3file, new File(localFile)).waitForCompletion(); + } catch (InterruptedException e) { + System.out.println("Transfer interrupted"); + } finally { + tx.shutdownNow(); + } + } + + private static void downloadByFullPathAndFileNamePrefix(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3prefix = params.getRequired("s3prefix"); + final String localFolder = params.getRequired("localFolder"); + final String s3filePrefix = params.get("s3filePrefix", ""); + TransferManager tx = TransferManagerBuilder.defaultTransferManager(); + Predicate keyPredicate = getKeyFilterByFileNamePrefix(s3filePrefix); + KeyFilter keyFilter = s3filePrefix.isEmpty() ? KeyFilter.INCLUDE_ALL : + objectSummary -> keyPredicate.test(objectSummary.getKey()); + try { + tx.downloadDirectory(bucket, s3prefix, new File(localFolder), keyFilter).waitForCompletion(); + } catch (InterruptedException e) { + System.out.println("Transfer interrupted"); + } finally { + tx.shutdownNow(); + } + } + + private static Predicate getKeyFilterByFileNamePrefix(String s3filePrefix) { + if (s3filePrefix.isEmpty()) { + return key -> true; + } else { + return key -> { + String[] parts = key.split("/"); + String fileName = parts[parts.length - 1]; + return fileName.startsWith(s3filePrefix); + }; + } + } + + private static void deleteFile(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3file = params.getRequired("s3file"); + AmazonS3ClientBuilder.defaultClient().deleteObject(bucket, s3file); + } + + private static void deleteByFullPathPrefix(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3prefix = params.getRequired("s3prefix"); + String[] keys = listByFullPathPrefix(bucket, s3prefix).toArray(new String[] {}); + if (keys.length > 0) { + DeleteObjectsRequest request = new DeleteObjectsRequest(bucket).withKeys(keys); + AmazonS3ClientBuilder.defaultClient().deleteObjects(request); + } + } + + private static void numberOfLinesInFile(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3file = params.getRequired("s3file"); + AmazonS3 s3client = AmazonS3ClientBuilder.defaultClient(); + System.out.print(S3QueryUtil.queryFile(s3client, bucket, s3file, countQuery)); + s3client.shutdown(); + } + + private static void numberOfLinesInFilesWithFullAndNamePrefix(ParameterTool params) { + final String bucket = params.getRequired("bucket"); + final String s3prefix = params.getRequired("s3prefix"); + final String s3filePrefix = params.get("s3filePrefix", ""); + int parallelism = params.getInt("parallelism", 10); + + List files = listByFullPathPrefix(bucket, s3prefix); + + ExecutorService executor = Executors.newFixedThreadPool(parallelism); + AmazonS3 s3client = AmazonS3ClientBuilder.defaultClient(); + List> requests = + submitLineCountingRequestsForFilesAsync(executor, s3client, bucket, files, s3filePrefix); + int count = waitAndComputeTotalLineCountResult(requests); + + executor.shutdownNow(); + s3client.shutdown(); + System.out.print(count); + } + + private static List> submitLineCountingRequestsForFilesAsync( + ExecutorService executor, AmazonS3 s3client, String bucket, List files, String s3filePrefix) { + List> requests = new ArrayList<>(); + Predicate keyPredicate = getKeyFilterByFileNamePrefix(s3filePrefix); + files.forEach(file -> { + if (keyPredicate.test(file)) { + CompletableFuture result = new CompletableFuture<>(); + executor.execute(() -> + result.complete(Integer.parseInt(S3QueryUtil.queryFile(s3client, bucket, file, countQuery)))); + requests.add(result); + } + }); + return requests; + } + + private static int waitAndComputeTotalLineCountResult(List> requests) { + int count = 0; + for (CompletableFuture result : requests) { + try { + count += result.get(); + } catch (Throwable e) { + System.out.println("Failed count lines"); + e.printStackTrace(); + } + } + return count; + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 6b31881f684d6..e07cf22fa3d87 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -55,6 +55,7 @@ under the License. flink-sql-client-test flink-streaming-file-sink-test flink-state-evolution-test + flink-e2e-test-utils diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 6a6c8b6729b4a..832bdeef93a43 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -101,6 +101,7 @@ run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_b run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh" "skip_check_exceptions" run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh" "skip_check_exceptions" run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh" "skip_check_exceptions" +run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false false" "skip_check_exceptions" diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index bdf6f64c5a8f2..4f628fc6d1240 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -462,43 +462,6 @@ function check_result_hash { fi } -function s3_put { - local_file=$1 - bucket=$2 - s3_file=$3 - resource="/${bucket}/${s3_file}" - contentType="application/octet-stream" - dateValue=`date -R` - stringToSign="PUT\n\n${contentType}\n${dateValue}\n${resource}" - s3Key=$ARTIFACTS_AWS_ACCESS_KEY - s3Secret=$ARTIFACTS_AWS_SECRET_KEY - signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` - curl -X PUT -T "${local_file}" \ - -H "Host: ${bucket}.s3.amazonaws.com" \ - -H "Date: ${dateValue}" \ - -H "Content-Type: ${contentType}" \ - -H "Authorization: AWS ${s3Key}:${signature}" \ - https://${bucket}.s3.amazonaws.com/${s3_file} -} - -function s3_delete { - bucket=$1 - s3_file=$2 - resource="/${bucket}/${s3_file}" - contentType="application/octet-stream" - dateValue=`date -R` - stringToSign="DELETE\n\n${contentType}\n${dateValue}\n${resource}" - s3Key=$ARTIFACTS_AWS_ACCESS_KEY - s3Secret=$ARTIFACTS_AWS_SECRET_KEY - signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` - curl -X DELETE \ - -H "Host: ${bucket}.s3.amazonaws.com" \ - -H "Date: ${dateValue}" \ - -H "Content-Type: ${contentType}" \ - -H "Authorization: AWS ${s3Key}:${signature}" \ - https://${bucket}.s3.amazonaws.com/${s3_file} -} - # This function starts the given number of task managers and monitors their processes. # If a task manager process goes away a replacement is started. function tm_watchdog { @@ -692,3 +655,20 @@ function expect_in_taskmanager_logs { fi done } + +function wait_for_restart_to_complete { + local base_num_restarts=$1 + local jobid=$2 + + local current_num_restarts=${base_num_restarts} + local expected_num_restarts=$((current_num_restarts + 1)) + + echo "Waiting for restart to happen" + while ! [[ ${current_num_restarts} -eq ${expected_num_restarts} ]]; do + sleep 5 + current_num_restarts=$(get_job_metric ${jobid} "fullRestarts") + if [[ -z ${current_num_restarts} ]]; then + current_num_restarts=${base_num_restarts} + fi + done +} diff --git a/flink-end-to-end-tests/test-scripts/common_s3.sh b/flink-end-to-end-tests/test-scripts/common_s3.sh new file mode 100644 index 0000000000000..5c16bb75bea43 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/common_s3.sh @@ -0,0 +1,240 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then + echo "Did not find AWS environment variables, NOT running the e2e test." + exit 0 +else + echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running the e2e test." +fi + +if [[ -z "$ARTIFACTS_AWS_ACCESS_KEY" ]]; then + echo "Did not find AWS environment variables, NOT running the e2e test." + exit 0 +else + echo "Found AWS access key $ARTIFACTS_AWS_ACCESS_KEY, running the e2e test." +fi + +if [[ -z "$ARTIFACTS_AWS_SECRET_KEY" ]]; then + echo "Did not find AWS environment variables, NOT running the e2e test." + exit 0 +else + echo "Found AWS secret key $ARTIFACTS_AWS_SECRET_KEY, running the e2e test." +fi + +AWS_REGION="${AWS_REGION:-eu-west-1}" +AWS_ACCESS_KEY=$ARTIFACTS_AWS_ACCESS_KEY +AWS_SECRET_KEY=$ARTIFACTS_AWS_SECRET_KEY + +s3util="java -jar ${END_TO_END_DIR}/flink-e2e-test-utils/target/S3UtilProgram.jar" + +################################### +# Setup Flink s3 access. +# +# Globals: +# FLINK_DIR +# ARTIFACTS_AWS_ACCESS_KEY +# ARTIFACTS_AWS_SECRET_KEY +# Arguments: +# None +# Returns: +# None +################################### +function s3_setup { + # make sure we delete the file at the end + function s3_cleanup { + rm $FLINK_DIR/lib/flink-s3-fs*.jar + + # remove any leftover settings + sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" + sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" + } + trap s3_cleanup EXIT + + cp $FLINK_DIR/opt/flink-s3-fs-hadoop-*.jar $FLINK_DIR/lib/ + echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" + echo "s3.secret-key: $ARTIFACTS_AWS_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" +} + +s3_setup + +################################### +# List s3 objects by full path prefix. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - s3 full path key prefix +# Returns: +# List of s3 object keys, separated by newline +################################### +function s3_list { + AWS_REGION=$AWS_REGION \ + ${s3util} --action listByFullPathPrefix --s3prefix "$1" --bucket $ARTIFACTS_AWS_BUCKET +} + +################################### +# Download s3 object. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - local path to save file +# $2 - s3 object key +# Returns: +# None +################################### +function s3_get { + AWS_REGION=$AWS_REGION \ + ${s3util} --action downloadFile --localFile "$1" --s3file "$2" --bucket $ARTIFACTS_AWS_BUCKET +} + +################################### +# Download s3 objects to folder by full path prefix. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - local path to save folder with files +# $2 - s3 key full path prefix +# $3 - s3 file name prefix w/o directory to filter files by name (optional) +# Returns: +# None +################################### +function s3_get_by_full_path_and_filename_prefix { + local file_prefix="${3-}" + AWS_REGION=$AWS_REGION \ + ${s3util} --action downloadByFullPathAndFileNamePrefix \ + --localFolder "$1" --s3prefix "$2" --s3filePrefix "${file_prefix}" --bucket $ARTIFACTS_AWS_BUCKET +} + +################################### +# Upload file to s3 object. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - local file to upload +# $2 - s3 bucket +# $3 - s3 object key +# Returns: +# None +################################### +function s3_put { + local_file=$1 + bucket=$2 + s3_file=$3 + resource="/${bucket}/${s3_file}" + contentType="application/octet-stream" + dateValue=`date -R` + stringToSign="PUT\n\n${contentType}\n${dateValue}\n${resource}" + s3Key=$ARTIFACTS_AWS_ACCESS_KEY + s3Secret=$ARTIFACTS_AWS_SECRET_KEY + signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` + curl -X PUT -T "${local_file}" \ + -H "Host: ${bucket}.s3.amazonaws.com" \ + -H "Date: ${dateValue}" \ + -H "Content-Type: ${contentType}" \ + -H "Authorization: AWS ${s3Key}:${signature}" \ + https://${bucket}.s3.amazonaws.com/${s3_file} +} + +################################### +# Delete s3 object. +# +# Globals: +# None +# Arguments: +# $1 - s3 bucket +# $2 - s3 object key +# $3 - (optional) s3 host suffix +# Returns: +# None +################################### +function s3_delete { + bucket=$1 + s3_file=$2 + resource="/${bucket}/${s3_file}" + contentType="application/octet-stream" + dateValue=`date -R` + stringToSign="DELETE\n\n${contentType}\n${dateValue}\n${resource}" + s3Key=$ARTIFACTS_AWS_ACCESS_KEY + s3Secret=$ARTIFACTS_AWS_SECRET_KEY + signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` + curl -X DELETE \ + -H "Host: ${bucket}.s3.amazonaws.com" \ + -H "Date: ${dateValue}" \ + -H "Content-Type: ${contentType}" \ + -H "Authorization: AWS ${s3Key}:${signature}" \ + https://${bucket}.s3.amazonaws.com/${s3_file} +} + +################################### +# Delete s3 objects by full path prefix. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - s3 key full path prefix +# Returns: +# None +################################### +function s3_delete_by_full_path_prefix { + AWS_REGION=$AWS_REGION \ + ${s3util} --action deleteByFullPathPrefix --s3prefix "$1" --bucket $ARTIFACTS_AWS_BUCKET +} + +################################### +# Count number of lines in a file of s3 object. +# The lines has to be simple to comply with CSV format +# because SQL is used to query the s3 object. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - s3 file object key +# $2 - s3 bucket +# Returns: +# None +################################### +function s3_get_number_of_lines_in_file { + AWS_REGION=$AWS_REGION \ + ${s3util} --action numberOfLinesInFile --s3file "$1" --bucket $ARTIFACTS_AWS_BUCKET +} + +################################### +# Count number of lines in files of s3 objects filtered by prefix. +# The lines has to be simple to comply with CSV format +# because SQL is used to query the s3 objects. +# +# Globals: +# ARTIFACTS_AWS_BUCKET +# Arguments: +# $1 - s3 key prefix +# $2 - s3 bucket +# $3 - s3 file name prefix w/o directory to filter files by name (optional) +# Returns: +# None +################################### +function s3_get_number_of_lines_by_prefix { + local file_prefix="${3-}" + AWS_REGION=$AWS_REGION \ + ${s3util} --action numberOfLinesInFilesWithFullAndNamePrefix \ + --s3prefix "$1" --s3filePrefix "${file_prefix}" --bucket $ARTIFACTS_AWS_BUCKET +} diff --git a/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh index e5ac5bcecdce9..3d8386758526a 100755 --- a/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh +++ b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh @@ -19,33 +19,18 @@ # Tests for our shaded/bundled Hadoop S3A file system. -if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then - echo "Did not find AWS environment variables, NOT running Shaded Hadoop S3A e2e tests." - exit 0 -else - echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running Shaded Hadoop S3A e2e tests." -fi - source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/common_s3.sh s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a # make sure we delete the file at the end -function s3_cleanup { +function shaded_s3a_cleanup { s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a - rm $FLINK_DIR/lib/flink-s3-fs*.jar - - # remove any leftover settings - sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" - sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" } -trap s3_cleanup EXIT - -cp $FLINK_DIR/opt/flink-s3-fs-hadoop-*.jar $FLINK_DIR/lib/ -echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" -echo "s3.secret-key: $ARTIFACTS_AWS_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" +trap shaded_s3a_cleanup EXIT start_cluster $FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out -check_result_hash "WordCountWithShadedS3A" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" \ No newline at end of file +check_result_hash "WordCountWithShadedS3A" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" diff --git a/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh index 4092805dc278f..bd33b410dfd4a 100755 --- a/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh +++ b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh @@ -19,33 +19,18 @@ # Tests for our shaded/bundled Hadoop S3A file system. -if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then - echo "Did not find AWS environment variables, NOT running Shaded Presto S3 e2e tests." - exit 0 -else - echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running Shaded Presto S3 e2e tests." -fi - source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/common_s3.sh s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 # make sure we delete the file at the end -function s3_cleanup { +function shaded_presto_s3_cleanup { s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 - rm $FLINK_DIR/lib/flink-s3-fs*.jar } -trap s3_cleanup EXIT - -cp $FLINK_DIR/opt/flink-s3-fs-presto-*.jar $FLINK_DIR/lib/ -echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" -echo "s3.secret-key: $ARTIFACTS_AWS_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" +trap shaded_presto_s3_cleanup EXIT start_cluster $FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out check_result_hash "WordCountWithShadedPrestoS3" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" - -# remove any leftover settings -sed -i -e 's/s3.access-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" -sed -i -e 's/s3.secret-key: .*//' "$FLINK_DIR/conf/flink-conf.yaml" diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh index 17389ad6f4ec2..50f5afc131256 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh @@ -17,27 +17,37 @@ # limitations under the License. ################################################################################ -source "$(dirname "$0")"/common.sh - -TEST_PROGRAM_JAR="${END_TO_END_DIR}/flink-streaming-file-sink-test/target/StreamingFileSinkProgram.jar" +OUT_TYPE="${1:-local}" # other type: s3 -OUTPUT_PATH="$TEST_DATA_DIR/out" - -function wait_for_restart { - local base_num_restarts=$1 - - local current_num_restarts=${base_num_restarts} - local expected_num_restarts=$((current_num_restarts + 1)) +source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/common_s3.sh + +OUT=out +OUTPUT_PATH="$TEST_DATA_DIR/$OUT" +S3_OUTPUT_PATH="s3://$ARTIFACTS_AWS_BUCKET/$OUT" + +mkdir -p $OUTPUT_PATH + +if [ "${OUT_TYPE}" == "local" ]; then + echo "Use local output" + JOB_OUTPUT_PATH=${OUTPUT_PATH} +elif [ "${OUT_TYPE}" == "s3" ]; then + echo "Use s3 output" + JOB_OUTPUT_PATH=${S3_OUTPUT_PATH} +else + echo "Unknown output type: ${OUT_TYPE}" + exit 1 +fi - echo "Waiting for restart to happen" - while ! [[ ${current_num_restarts} -eq ${expected_num_restarts} ]]; do - sleep 5 - current_num_restarts=$(get_job_metric ${JOB_ID} "fullRestarts") - if [[ -z ${current_num_restarts} ]]; then - current_num_restarts=${base_num_restarts} - fi - done +# make sure we delete the file at the end +function out_cleanup { + s3_delete_by_full_path_prefix $OUT } +if [ "${OUT_TYPE}" == "s3" ]; then + trap out_cleanup EXIT +fi + +TEST_PROGRAM_JAR="${END_TO_END_DIR}/flink-streaming-file-sink-test/target/StreamingFileSinkProgram.jar" ################################### # Get all lines in part files and sort them numerically. @@ -47,10 +57,32 @@ function wait_for_restart { # Arguments: # None # Returns: -# None +# sorted content of part files ################################### function get_complete_result { - find "${OUTPUT_PATH}" -type f \( -iname "part-*" \) -exec cat {} + | sort -g + if [ "${OUT_TYPE}" == "s3" ]; then + rm -rf $OUTPUT_PATH; mkdir -p $OUTPUT_PATH + s3_get_by_full_path_and_filename_prefix ${TEST_DATA_DIR} "${OUT}" "part-" + fi + find "${OUTPUT_PATH}" -type f \( -iname "part-*" \) -exec cat {} + | sort -g +} + +################################### +# Get total number of lines in part files. +# +# Globals: +# OUT +# Arguments: +# None +# Returns: +# line number in part files +################################### +function get_total_number_of_valid_lines { + if [ "${OUT_TYPE}" == "local" ]; then + get_complete_result | wc -l | tr -d '[:space:]' + elif [ "${OUT_TYPE}" == "s3" ]; then + s3_get_number_of_lines_by_prefix "${OUT}" "part-" + fi } ################################### @@ -83,7 +115,7 @@ function wait_for_complete_result { sleep ${polling_interval} ((seconds_elapsed += ${polling_interval})) - number_of_values=$(get_complete_result | wc -l | tr -d '[:space:]') + number_of_values=$(get_total_number_of_valid_lines) if [[ ${previous_number_of_values} -ne ${number_of_values} ]]; then echo "Number of produced values ${number_of_values}/${expected_number_of_values}" previous_number_of_values=${number_of_values} @@ -98,7 +130,7 @@ start_cluster "${FLINK_DIR}/bin/taskmanager.sh" start echo "Submitting job." -CLIENT_OUTPUT=$("$FLINK_DIR/bin/flink" run -d "${TEST_PROGRAM_JAR}" --outputPath "${OUTPUT_PATH}") +CLIENT_OUTPUT=$("$FLINK_DIR/bin/flink" run -d "${TEST_PROGRAM_JAR}" --outputPath "${JOB_OUTPUT_PATH}") JOB_ID=$(echo "${CLIENT_OUTPUT}" | grep "Job has been submitted with JobID" | sed 's/.* //g') if [[ -z $JOB_ID ]]; then @@ -117,7 +149,7 @@ kill_random_taskmanager echo "Starting TM" "$FLINK_DIR/bin/taskmanager.sh" start -wait_for_restart 0 +wait_for_restart_to_complete 0 ${JOB_ID} echo "Killing 2 TMs" kill_random_taskmanager @@ -127,7 +159,7 @@ echo "Starting 2 TMs" "$FLINK_DIR/bin/taskmanager.sh" start "$FLINK_DIR/bin/taskmanager.sh" start -wait_for_restart 1 +wait_for_restart_to_complete 1 ${JOB_ID} echo "Waiting until all values have been produced" wait_for_complete_result 60000 300 From 18c157970edefececbd040c438ae25cc44d47c42 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 5 Nov 2018 11:39:51 +0100 Subject: [PATCH 008/359] [FLINK-10793][ttl] Change visibility of TtlValue and TtlSerializer to public for external tools This closes #7021. --- .../flink/runtime/state/ttl/TtlStateFactory.java | 10 ++++++---- .../org/apache/flink/runtime/state/ttl/TtlValue.java | 10 +++++----- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java index 45f4e3bf143dd..0a881c709781c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlStateFactory.java @@ -188,15 +188,17 @@ private StateSnapshotTransformFactory getSnapshotTransformFactory() { } } - /** Serializer for user state value with TTL. */ - private static class TtlSerializer extends CompositeSerializer> { + /** + * Serializer for user state value with TTL. Visibility is public for usage with external tools. + */ + public static class TtlSerializer extends CompositeSerializer> { private static final long serialVersionUID = 131020282727167064L; - TtlSerializer(TypeSerializer userValueSerializer) { + public TtlSerializer(TypeSerializer userValueSerializer) { super(true, LongSerializer.INSTANCE, userValueSerializer); } - TtlSerializer(PrecomputedParameters precomputed, TypeSerializer ... fieldSerializers) { + public TtlSerializer(PrecomputedParameters precomputed, TypeSerializer ... fieldSerializers) { super(precomputed, fieldSerializers); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlValue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlValue.java index 48435d567e86e..dba06d3a88b0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlValue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlValue.java @@ -23,28 +23,28 @@ import java.io.Serializable; /** - * This class wraps user value of state with TTL. + * This class wraps user value of state with TTL. Visibility is public for usage with external tools. * * @param Type of the user value of state with TTL */ -class TtlValue implements Serializable { +public class TtlValue implements Serializable { private static final long serialVersionUID = 5221129704201125020L; @Nullable private final T userValue; private final long lastAccessTimestamp; - TtlValue(@Nullable T userValue, long lastAccessTimestamp) { + public TtlValue(@Nullable T userValue, long lastAccessTimestamp) { this.userValue = userValue; this.lastAccessTimestamp = lastAccessTimestamp; } @Nullable - T getUserValue() { + public T getUserValue() { return userValue; } - long getLastAccessTimestamp() { + public long getLastAccessTimestamp() { return lastAccessTimestamp; } } From b7a76e0afd12b8c84c5e3a580e77fe6b215f150b Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 31 Oct 2018 15:27:47 +0100 Subject: [PATCH 009/359] [FLINK-8897] [table] Reintroduce materialization of time attributes in filters --- .../calcite/RelTimeIndicatorConverter.scala | 82 +++++++++++-------- .../rules/datastream/DataStreamJoinRule.scala | 2 +- .../datastream/DataStreamWindowJoinRule.scala | 2 +- .../table/runtime/join/WindowJoinUtil.scala | 12 +-- .../flink/table/api/stream/sql/JoinTest.scala | 72 ++++++++-------- .../table/api/stream/table/JoinTest.scala | 39 +++++---- .../plan/TimeIndicatorConversionTest.scala | 2 +- .../runtime/stream/TimeAttributesITCase.scala | 46 +++++++++++ 8 files changed, 168 insertions(+), 89 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala index 41f0fc5f685c4..c1bcf1438d354 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala @@ -23,6 +23,7 @@ import org.apache.calcite.rel.core._ import org.apache.calcite.rel.logical._ import org.apache.calcite.rel.{RelNode, RelShuttle} import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo import org.apache.flink.table.api.{TableException, ValidationException} @@ -100,13 +101,7 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { override def visit(matchRel: LogicalMatch): RelNode = { // visit children and update inputs val input = matchRel.getInput.accept(this) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - input.getRowType.getFieldList.map(_.getType)) + val materializer = createMaterializer(input) // update input expressions val patternDefs = matchRel.getPatternDefinitions.mapValues(_.accept(materializer)) @@ -180,23 +175,16 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { override def visit(filter: LogicalFilter): RelNode = { // visit children and update inputs val input = filter.getInput.accept(this) + val materializer = createMaterializer(input) - // We do not materialize time indicators in conditions because they can be locally evaluated. - // Some conditions are evaluated by special operators (e.g., time window joins). - // Time indicators in remaining conditions are materialized by Calc before the code generation. - LogicalFilter.create(input, filter.getCondition) + val condition = filter.getCondition.accept(materializer) + LogicalFilter.create(input, condition) } override def visit(project: LogicalProject): RelNode = { // visit children and update inputs val input = project.getInput.accept(this) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - input.getRowType.getFieldList.map(_.getType)) + val materializer = createMaterializer(input) val projects = project.getProjects.map(_.accept(materializer)) val fieldNames = project.getRowType.getFieldNames @@ -206,8 +194,14 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { override def visit(join: LogicalJoin): RelNode = { val left = join.getLeft.accept(this) val right = join.getRight.accept(this) + val materializer = createMaterializer(left, right) - LogicalJoin.create(left, right, join.getCondition, join.getVariablesSet, join.getJoinType) + LogicalJoin.create( + left, + right, + join.getCondition.accept(materializer), + join.getVariablesSet, + join.getJoinType) } def visit(temporalJoin: LogicalTemporalTableJoin): RelNode = { @@ -229,19 +223,11 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { case scan: LogicalTableFunctionScan => // visit children and update inputs val scanInputs = scan.getInputs.map(_.accept(this)) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - inputs.head.getRowType.getFieldList.map(_.getType)) - - val call = scan.getCall.accept(materializer) + val materializer = createMaterializer(inputs.head) LogicalTableFunctionScan.create( scan.getCluster, scanInputs, - call, + scan.getCall.accept(materializer), scan.getElementType, scan.getRowType, scan.getColumnMappings) @@ -369,6 +355,15 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { indicesToMaterialize.toSet } + + private def createMaterializer(inputs: RelNode*): RexTimeIndicatorMaterializer = { + // check if input field contains time indicator type + // materialize field if no time indicator is present anymore + // if input field is already materialized, change to timestamp type + new RexTimeIndicatorMaterializer( + rexBuilder, + inputs.flatMap(_.getRowType.getFieldList.map(_.getType))) + } } object RelTimeIndicatorConverter { @@ -412,11 +407,34 @@ object RelTimeIndicatorConverter { * @return The expression with materialized time indicators. */ def convertExpression(expr: RexNode, rowType: RelDataType, rexBuilder: RexBuilder): RexNode = { + // check if input field contains time indicator type + // materialize field if no time indicator is present anymore + // if input field is already materialized, change to timestamp type val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - rowType.getFieldList.map(_.getType)) + rexBuilder, + rowType.getFieldList.map(_.getType)) + + expr.accept(materializer) + } + + /** + * Checks if the given call is a materialization call for either proctime or rowtime. + */ + def isMaterializationCall(call: RexCall): Boolean = { + val isProctimeCall: Boolean = { + call.getOperator == ProctimeSqlFunction && + call.getOperands.size() == 1 && + isProctimeIndicatorType(call.getOperands.get(0).getType) + } + + val isRowtimeCall: Boolean = { + call.getOperator == SqlStdOperatorTable.CAST && + call.getOperands.size() == 1 && + isRowtimeIndicatorType(call.getOperands.get(0).getType) && + call.getType.getSqlTypeName == SqlTypeName.TIMESTAMP + } - expr.accept(materializer) + isProctimeCall || isRowtimeCall } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala index 072acb34c7309..f51c08888eb34 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala @@ -73,7 +73,7 @@ class DataStreamJoinRule val remainingPredsAccessTime = remainingPreds.isDefined && accessesTimeAttribute(remainingPreds.get, join.getRowType) - // Check that no event-time attributes are in the input because non-window join is unbounded + // Check that no event-time attributes are in the output because non-window join is unbounded // and we don't know how much to hold back watermarks. val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala index 3dfae99a69629..cd9c5a86f342c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala @@ -52,7 +52,7 @@ class DataStreamWindowJoinRule if (windowBounds.get.isEventTime) { true } else { - // Check that no event-time attributes are in the input because the processing time window + // Check that no event-time attributes are in the output because the processing time window // join does not correctly hold back watermarks. // We rely on projection pushdown to remove unused attributes before the join. !join.getRowType.getFieldList.asScala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala index 18e26df89ccab..3e355e800a10a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala @@ -23,12 +23,14 @@ import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rex._ -import org.apache.calcite.sql.SqlKind +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.sql.{SqlKind, SqlOperatorTable} import org.apache.flink.api.common.functions.FlatJoinFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.calcite.{FlinkTypeFactory, RelTimeIndicatorConverter} import org.apache.flink.table.codegen.{ExpressionReducer, FunctionCodeGenerator, GeneratedFunction} +import org.apache.flink.table.functions.sql.ProctimeSqlFunction import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType} import org.apache.flink.types.Row @@ -380,13 +382,13 @@ object WindowJoinUtil { */ def replaceTimeFieldWithLiteral(expr: RexNode): RexNode = { expr match { + case c: RexCall if RelTimeIndicatorConverter.isMaterializationCall(c) => + // replace with timestamp + rexBuilder.makeZeroLiteral(expr.getType) case c: RexCall => // replace in call operands val newOps = c.operands.asScala.map(replaceTimeFieldWithLiteral).asJava rexBuilder.makeCall(c.getType, c.getOperator, newOps) - case i: RexInputRef if FlinkTypeFactory.isTimeIndicatorType(i.getType) => - // replace with timestamp - rexBuilder.makeZeroLiteral(expr.getType) case _ => expr } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index 736f9a2cad0c1..f435113029386 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -21,6 +21,7 @@ import org.apache.calcite.rel.logical.LogicalJoin import org.apache.flink.api.scala._ import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ +import org.apache.flink.table.calcite.RelTimeIndicatorConverter import org.apache.flink.table.expressions.Null import org.apache.flink.table.plan.logical.TumblingGroupWindow import org.apache.flink.table.runtime.join.WindowJoinUtil @@ -29,6 +30,9 @@ import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} import org.junit.Assert._ import org.junit.Test +/** + * Tests for both windowed and non-windowed joins. + */ class JoinTest extends TableTestBase { private val streamUtil: StreamTableTestUtil = streamTestUtil() streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 'proctime.proctime) @@ -62,8 +66,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "proctime") ), term("where", - "AND(=(a, a0), >=(proctime, -(proctime0, 3600000)), " + - "<=(proctime, +(proctime0, 3600000)))"), + "AND(=(a, a0), >=(PROCTIME(proctime), -(PROCTIME(proctime0), 3600000)), " + + "<=(PROCTIME(proctime), +(PROCTIME(proctime0), 3600000)))"), term("join", "a, proctime, a0, b, proctime0"), term("joinType", "InnerJoin") ), @@ -100,8 +104,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 10000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 10000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, c, a0, b, c0"), term("joinType", "InnerJoin") ), @@ -138,8 +142,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "proctime") ), term("where", - "AND(=(a, a0), >=(proctime, -(proctime0, 3600000)), " + - "<=(proctime, +(proctime0, 3600000)))"), + "AND(=(a, a0), >=(PROCTIME(proctime), -(PROCTIME(proctime0), 3600000)), " + + "<=(PROCTIME(proctime), +(PROCTIME(proctime0), 3600000)))"), term("join", "a, proctime, a0, b, proctime0"), term("joinType", "InnerJoin") ), @@ -176,8 +180,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 600000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 600000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, c, a0, b, c0"), term("joinType", "InnerJoin") ), @@ -208,7 +212,7 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "a", "b", "proctime") ), - term("where", "AND(=(a, a0), =(proctime, proctime0))"), + term("where", "AND(=(a, a0), =(PROCTIME(proctime), PROCTIME(proctime0)))"), term("join", "a", "proctime", "a0", "b", "proctime0"), term("joinType", "InnerJoin") ), @@ -238,7 +242,7 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "a", "b", "c") ), - term("where", "AND(=(a, a0), =(c, c0))"), + term("where", "AND(=(a, a0), =(CAST(c), CAST(c0)))"), term("join", "a", "c", "a0", "b", "c0"), term("joinType", "InnerJoin") ), @@ -280,8 +284,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "a", "c", "proctime", "12 AS nullField") ), - term("where", "AND(=(a, a0), =(nullField, nullField0), >=(proctime, " + - "-(proctime0, 5000)), <=(proctime, +(proctime0, 5000)))"), + term("where", "AND(=(a, a0), =(nullField, nullField0), >=(PROCTIME(proctime), " + + "-(PROCTIME(proctime0), 5000)), <=(PROCTIME(proctime), +(PROCTIME(proctime0), 5000)))"), term("join", "a", "c", "proctime", "nullField", "a0", "c0", "proctime0", "nullField0"), term("joinType", "InnerJoin") ), @@ -320,8 +324,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 600000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 600000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, b, c, a0, b0, c0"), term("joinType", "InnerJoin") ), @@ -365,8 +369,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 600000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 600000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, b, c, a0, b0, c0"), term("joinType", "InnerJoin") ), @@ -408,8 +412,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "proctime") ), term("where", - "AND(=(a, a0), >=(proctime, -(proctime0, 3600000)), " + - "<=(proctime, +(proctime0, 3600000)))"), + "AND(=(a, a0), >=(PROCTIME(proctime), -(PROCTIME(proctime0), 3600000)), " + + "<=(PROCTIME(proctime), +(PROCTIME(proctime0), 3600000)))"), term("join", "a, proctime, a0, b, proctime0"), term("joinType", "LeftOuterJoin") ), @@ -446,8 +450,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 10000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 10000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, c, a0, b, c0"), term("joinType", "LeftOuterJoin") ), @@ -485,8 +489,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "proctime") ), term("where", - "AND(=(a, a0), >=(proctime, -(proctime0, 3600000)), " + - "<=(proctime, +(proctime0, 3600000)))"), + "AND(=(a, a0), >=(PROCTIME(proctime), -(PROCTIME(proctime0), 3600000)), " + + "<=(PROCTIME(proctime), +(PROCTIME(proctime0), 3600000)))"), term("join", "a, proctime, a0, b, proctime0"), term("joinType", "RightOuterJoin") ), @@ -523,8 +527,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 10000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 10000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, c, a0, b, c0"), term("joinType", "RightOuterJoin") ), @@ -562,8 +566,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "proctime") ), term("where", - "AND(=(a, a0), >=(proctime, -(proctime0, 3600000)), " + - "<=(proctime, +(proctime0, 3600000)))"), + "AND(=(a, a0), >=(PROCTIME(proctime), -(PROCTIME(proctime0), 3600000)), " + + "<=(PROCTIME(proctime), +(PROCTIME(proctime0), 3600000)))"), term("join", "a, proctime, a0, b, proctime0"), term("joinType", "FullOuterJoin") ), @@ -600,8 +604,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 10000)), " + - "<=(c, +(c0, 3600000)))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 10000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)))"), term("join", "a, c, a0, b, c0"), term("joinType", "FullOuterJoin") ), @@ -640,8 +644,8 @@ class JoinTest extends TableTestBase { term("select", "a", "b", "c") ), term("where", - "AND(=(a, a0), >=(c, -(c0, 10000)), " + - "<=(c, +(c0, 3600000)), LIKE(b, b0))"), + "AND(=(a, a0), >=(CAST(c), -(CAST(c0), 10000)), " + + "<=(CAST(c), +(CAST(c0), 3600000)), LIKE(b, b0))"), term("join", "a, b, c, a0, b0, c0"), // Since we filter on attributes b and b0 after the join, the full outer join // will be automatically optimized to inner join. @@ -795,7 +799,9 @@ class JoinTest extends TableTestBase { "SELECT t1.a, t2.b FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " + timeSql val resultTable = streamUtil.tableEnv.sqlQuery(query) - val relNode = resultTable.getRelNode + val relNode = RelTimeIndicatorConverter.convert( + resultTable.getRelNode, + streamUtil.tableEnv.getRelBuilder.getRexBuilder) val joinNode = relNode.getInput(0).asInstanceOf[LogicalJoin] val (windowBounds, _) = WindowJoinUtil.extractWindowBoundsFromPredicate( @@ -1008,7 +1014,9 @@ class JoinTest extends TableTestBase { expectCondStr: String): Unit = { val resultTable = streamUtil.tableEnv.sqlQuery(query) - val relNode = resultTable.getRelNode + val relNode = RelTimeIndicatorConverter.convert( + resultTable.getRelNode, + streamUtil.tableEnv.getRelBuilder.getRexBuilder) val joinNode = relNode.getInput(0).asInstanceOf[LogicalJoin] val (_, remainCondition) = WindowJoinUtil.extractWindowBoundsFromPredicate( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala index d7f5c71581592..138497cada3dd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/JoinTest.scala @@ -27,7 +27,7 @@ import org.apache.flink.table.utils.TableTestUtil._ import org.junit.Test /** - * Currently only time-windowed joins can be processed in a streaming fashion. + * Tests for both windowed and non-windowed joins. */ class JoinTest extends TableTestBase { @@ -57,8 +57,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rrtime") ), - term("where", "AND(=(a, d), >=(lrtime, -(rrtime, 300000))," + - " <(lrtime, +(rrtime, 3000)))"), + term("where", "AND(=(a, d), >=(CAST(lrtime), -(CAST(rrtime), 300000))," + + " <(CAST(lrtime), +(CAST(rrtime), 3000)))"), term("join", "a", "lrtime", "d", "e", "rrtime"), term("joinType", "InnerJoin") ), @@ -92,7 +92,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rptime") ), - term("where", "AND(=(a, d), >=(lptime, -(rptime, 1000)), <(lptime, rptime))"), + term("where", "AND(=(a, d), >=(PROCTIME(lptime), -(PROCTIME(rptime), 1000)), " + + "<(PROCTIME(lptime), PROCTIME(rptime)))"), term("join", "a", "lptime", "d", "e", "rptime"), term("joinType", "InnerJoin") ), @@ -126,7 +127,7 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rptime") ), - term("where", "AND(=(a, d), =(lptime, rptime))"), + term("where", "AND(=(a, d), =(PROCTIME(lptime), PROCTIME(rptime)))"), term("join", "a", "lptime", "d", "e", "rptime"), term("joinType", "InnerJoin") ), @@ -153,7 +154,8 @@ class JoinTest extends TableTestBase { streamTableNode(0), streamTableNode(1), term("where", - "AND(=(a, d), >=(lrtime, -(rrtime, 300000)), <(lrtime, rrtime), >(lrtime, " + "f))"), + "AND(=(a, d), >=(CAST(lrtime), -(CAST(rrtime), 300000)), " + + "<(CAST(lrtime), CAST(rrtime)), >(CAST(lrtime), f))"), term("join", "a", "b", "c", "lrtime", "d", "e", "f", "rrtime"), term("joinType", "InnerJoin") ) @@ -188,8 +190,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rrtime") ), - term("where", "AND(=(a, d), >=(lrtime, -(rrtime, 300000))," + - " <(lrtime, +(rrtime, 3000)))"), + term("where", "AND(=(a, d), >=(CAST(lrtime), -(CAST(rrtime), 300000))," + + " <(CAST(lrtime), +(CAST(rrtime), 3000)))"), term("join", "a", "lrtime", "d", "e", "rrtime"), term("joinType", "LeftOuterJoin") ), @@ -223,7 +225,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rptime") ), - term("where", "AND(=(a, d), >=(lptime, -(rptime, 1000)), <(lptime, rptime))"), + term("where", "AND(=(a, d), >=(PROCTIME(lptime), -(PROCTIME(rptime), 1000)), " + + "<(PROCTIME(lptime), PROCTIME(rptime)))"), term("join", "a", "lptime", "d", "e", "rptime"), term("joinType", "LeftOuterJoin") ), @@ -260,8 +263,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rrtime") ), - term("where", "AND(=(a, d), >=(lrtime, -(rrtime, 300000))," + - " <(lrtime, +(rrtime, 3000)))"), + term("where", "AND(=(a, d), >=(CAST(lrtime), -(CAST(rrtime), 300000))," + + " <(CAST(lrtime), +(CAST(rrtime), 3000)))"), term("join", "a", "lrtime", "d", "e", "rrtime"), term("joinType", "RightOuterJoin") ), @@ -295,7 +298,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rptime") ), - term("where", "AND(=(a, d), >=(lptime, -(rptime, 1000)), <(lptime, rptime))"), + term("where", "AND(=(a, d), >=(PROCTIME(lptime), -(PROCTIME(rptime), 1000)), " + + "<(PROCTIME(lptime), PROCTIME(rptime)))"), term("join", "a", "lptime", "d", "e", "rptime"), term("joinType", "RightOuterJoin") ), @@ -332,8 +336,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rrtime") ), - term("where", "AND(=(a, d), >=(lrtime, -(rrtime, 300000))," + - " <(lrtime, +(rrtime, 3000)))"), + term("where", "AND(=(a, d), >=(CAST(lrtime), -(CAST(rrtime), 300000))," + + " <(CAST(lrtime), +(CAST(rrtime), 3000)))"), term("join", "a", "lrtime", "d", "e", "rrtime"), term("joinType", "FullOuterJoin") ), @@ -367,7 +371,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rptime") ), - term("where", "AND(=(a, d), >=(lptime, -(rptime, 1000)), <(lptime, rptime))"), + term("where", "AND(=(a, d), >=(PROCTIME(lptime), -(PROCTIME(rptime), 1000)), " + + "<(PROCTIME(lptime), PROCTIME(rptime)))"), term("join", "a", "lptime", "d", "e", "rptime"), term("joinType", "FullOuterJoin") ), @@ -402,8 +407,8 @@ class JoinTest extends TableTestBase { streamTableNode(1), term("select", "d", "e", "rrtime") ), - term("where", "AND(=(a, d), >=(lrtime, -(rrtime, 300000))," + - " <(lrtime, +(rrtime, 3000)))"), + term("where", "AND(=(a, d), >=(CAST(lrtime), -(CAST(rrtime), 300000))," + + " <(CAST(lrtime), +(CAST(rrtime), 3000)))"), term("join", "a", "lrtime", "d", "e", "rrtime"), // Since we filter on attributes of the left table after the join, the left outer join // will be automatically optimized to inner join. diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala index 1706169215398..29dda21f93226 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala @@ -85,7 +85,7 @@ class TimeIndicatorConversionTest extends TableTestBase { "DataStreamCalc", streamTableNode(0), term("select", "rowtime"), - term("where", ">(rowtime, 1990-12-02 12:11:11)") + term("where", ">(CAST(rowtime), 1990-12-02 12:11:11)") ) util.verifyTable(result, expected) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index 304dbb317f358..1706fc8a112e7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.runtime.stream import java.lang.{Integer => JInt, Long => JLong} import java.math.BigDecimal +import java.sql.Timestamp import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo @@ -661,6 +662,51 @@ class TimeAttributesITCase extends AbstractTestBase { ) assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + @Test + def testMaterializedRowtimeFilter(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setParallelism(1) + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val data = new mutable.MutableList[(String, Timestamp, Int)] + data.+=(("ACME", new Timestamp(1000L), 12)) + data.+=(("ACME", new Timestamp(2000L), 17)) + data.+=(("ACME", new Timestamp(3000L), 13)) + data.+=(("ACME", new Timestamp(4000L), 11)) + + val t = env.fromCollection(data) + .assignAscendingTimestamps(e => e._2.toInstant.toEpochMilli) + .toTable(tEnv, 'symbol, 'tstamp.rowtime, 'price) + tEnv.registerTable("Ticker", t) + + val sqlQuery = + s""" + |SELECT * + |FROM ( + | SELECT symbol, SUM(price) as price, + | TUMBLE_ROWTIME(tstamp, interval '1' second) as rowTime, + | TUMBLE_START(tstamp, interval '1' second) as startTime, + | TUMBLE_END(tstamp, interval '1' second) as endTime + | FROM Ticker + | GROUP BY symbol, TUMBLE(tstamp, interval '1' second) + |) + |WHERE startTime < endTime + |""".stripMargin + + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = List( + "ACME,12,1970-01-01 00:00:01.999,1970-01-01 00:00:01.0,1970-01-01 00:00:02.0", + "ACME,17,1970-01-01 00:00:02.999,1970-01-01 00:00:02.0,1970-01-01 00:00:03.0", + "ACME,13,1970-01-01 00:00:03.999,1970-01-01 00:00:03.0,1970-01-01 00:00:04.0", + "ACME,11,1970-01-01 00:00:04.999,1970-01-01 00:00:04.0,1970-01-01 00:00:05.0") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } } object TimeAttributesITCase { From 4d6d14543666e71130b723589f1239943ca118d4 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 5 Nov 2018 13:57:37 +0100 Subject: [PATCH 010/359] [hotfix] [table] Simplify time attribute handling for joins --- .../rules/datastream/DataStreamJoinRule.scala | 41 +++------ .../DataStreamTemporalTableJoinRule.scala | 13 +-- .../sql/validation/JoinValidationTest.scala | 88 ++++++++++++++++--- 3 files changed, 89 insertions(+), 53 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala index f51c08888eb34..ad749d25df32f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala @@ -20,10 +20,8 @@ package org.apache.flink.table.plan.rules.datastream import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.convert.ConverterRule -import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode} -import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin @@ -40,45 +38,32 @@ class DataStreamJoinRule FlinkConventions.DATASTREAM, "DataStreamJoinRule") { - /** - * Checks if an expression accesses a time attribute. - * - * @param expr The expression to check. - * @param inputType The input type of the expression. - * @return True, if the expression accesses a time attribute. False otherwise. - */ - def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = { - expr match { - case i: RexInputRef => - val accessedType = inputType.getFieldList.get(i.getIndex).getType - FlinkTypeFactory.isTimeIndicatorType(accessedType) - case c: RexCall => - c.operands.asScala.exists(accessesTimeAttribute(_, inputType)) - case _ => false - } - } - override def matches(call: RelOptRuleCall): Boolean = { val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] val joinInfo = join.analyzeCondition - val (windowBounds, remainingPreds) = WindowJoinUtil.extractWindowBoundsFromPredicate( + val (windowBounds, _) = WindowJoinUtil.extractWindowBoundsFromPredicate( joinInfo.getRemaining(join.getCluster.getRexBuilder), join.getLeft.getRowType.getFieldCount, join.getRowType, join.getCluster.getRexBuilder, TableConfig.DEFAULT) - // remaining predicate must not access time attributes - val remainingPredsAccessTime = remainingPreds.isDefined && - accessesTimeAttribute(remainingPreds.get, join.getRowType) + if (windowBounds.isDefined) { + return false + } - // Check that no event-time attributes are in the output because non-window join is unbounded - // and we don't know how much to hold back watermarks. + // Check that no event-time attributes are in the outputs (composed of two inputs) + // because non-window join is unbounded and we don't know how much to hold back watermarks. val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) - windowBounds.isEmpty && !remainingPredsAccessTime && !rowTimeAttrInOutput + if (rowTimeAttrInOutput) { + throw new TableException( + "Rowtime attributes must not be in the input rows of a regular join. " + + "As a workaround you can cast the time attributes of input tables to TIMESTAMP before.") + } + true } override def convert(rel: RelNode): RelNode = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala index 94ff19c229daf..5bdea53b58a27 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala @@ -22,12 +22,10 @@ import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.JoinRelType -import org.apache.flink.table.api.TableConfig import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.DataStreamTemporalTableJoin import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTemporalTableJoin import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.table.runtime.join.WindowJoinUtil class DataStreamTemporalTableJoinRule extends ConverterRule( @@ -38,16 +36,7 @@ class DataStreamTemporalTableJoinRule override def matches(call: RelOptRuleCall): Boolean = { val join: FlinkLogicalTemporalTableJoin = call.rel(0) - val joinInfo = join.analyzeCondition - - val (windowBounds, remainingPreds) = WindowJoinUtil.extractWindowBoundsFromPredicate( - joinInfo.getRemaining(join.getCluster.getRexBuilder), - join.getLeft.getRowType.getFieldCount, - join.getRowType, - join.getCluster.getRexBuilder, - TableConfig.DEFAULT) - - windowBounds.isEmpty && join.getJoinType == JoinRelType.INNER + join.getJoinType == JoinRelType.INNER } override def convert(rel: RelNode): RelNode = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala index 141c8173ea049..2a994f31a2c64 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala @@ -19,8 +19,8 @@ package org.apache.flink.table.api.stream.sql.validation import org.apache.flink.api.scala._ -import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.TableException +import org.apache.flink.table.api.scala._ import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} import org.junit.Test @@ -30,18 +30,6 @@ class JoinValidationTest extends TableTestBase { streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 'proctime.proctime) streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 'proctime.proctime) - /** There should exist exactly two time conditions **/ - @Test(expected = classOf[TableException]) - def testWindowJoinSingleTimeCondition() = { - val sql = - """ - |SELECT t2.a - |FROM MyTable t1 JOIN MyTable2 t2 ON - | t1.a = t2.a AND - | t1.proctime > t2.proctime - INTERVAL '5' SECOND""".stripMargin - streamUtil.verifySql(sql, "n/a") - } - /** Both time attributes in a join condition must be of the same type **/ @Test(expected = classOf[TableException]) def testWindowJoinDiffTimeIndicator() = { @@ -136,4 +124,78 @@ class JoinValidationTest extends TableTestBase { streamUtil.verifySql(sql, "n/a") } + + /** Rowtime attributes cannot be accessed in filter conditions yet. */ + @Test + def testJoinWithRowtimeCondition(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage( + "Rowtime attributes must not be in the input rows of a regular join.") + + val sql = + """ + |SELECT t2.a + |FROM MyTable t1 JOIN MyTable2 t2 ON + | t1.a = t2.a AND + | t1.c > t2.c - INTERVAL '5' SECOND + |""".stripMargin + + streamUtil.verifySql(sql, "n/a") + } + + /** Rowtime attributes cannot be accessed in filter conditions yet. */ + @Test + def testJoinWithRowtimeConditionFromComplexQuery(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage( + "Rowtime attributes must not be in the input rows of a regular join.") + + val util = streamTestUtil() + + util.addTable[(Long, Long)]("MyTable1", 'id, 'eventTs.rowtime) + + util.addTable[(Long, Long)]("MyTable2", 'id, 'eventTs.rowtime) + + val sql1 = + """SELECT + | id, + | eventTs AS t1, + | COUNT(*) OVER ( + | PARTITION BY id ORDER BY eventTs ROWS BETWEEN 100 PRECEDING AND CURRENT ROW + | ) AS cnt1 + |FROM MyTable1 + |""".stripMargin + val sql2 = + """SELECT DISTINCT + | id AS r_id, + | eventTs AS t2, + | COUNT(*) OVER ( + | PARTITION BY id ORDER BY eventTs ROWS BETWEEN 50 PRECEDING AND CURRENT ROW + | ) AS cnt2 + |FROM MyTable2 + |""".stripMargin + + val left = util.tableEnv.sqlQuery(sql1) + val right = util.tableEnv.sqlQuery(sql2) + val result = left.join(right).where("id === r_id && t1 === t2").select("id, t1") + + util.verifyTable(result, "n/a") + } + + /** Rowtime attributes cannot be accessed in projection yet. */ + @Test + def testJoinWithRowtimeProjection(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage( + "Rowtime attributes must not be in the input rows of a regular join.") + + val sql = + """ + |SELECT t2.a, t2.c + |FROM MyTable t1 JOIN MyTable2 t2 ON + | t1.a = t2.a + |""".stripMargin + + streamUtil.verifySql(sql, "n/a") + } } From 9c0fddecb15536e4624c5b88af91ce19ee5ff622 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 5 Nov 2018 13:58:12 +0100 Subject: [PATCH 011/359] [hotfix] [table] Move utility method down in JoinTest --- .../flink/table/api/stream/sql/JoinTest.scala | 58 +++++++++---------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index f435113029386..37d5bc1ea9795 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -790,35 +790,6 @@ class JoinTest extends TableTestBase { "AND(=($0, $4), >($2, $6))") } - private def verifyTimeBoundary( - timeSql: String, - expLeftSize: Long, - expRightSize: Long, - expTimeType: String): Unit = { - val query = - "SELECT t1.a, t2.b FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " + timeSql - - val resultTable = streamUtil.tableEnv.sqlQuery(query) - val relNode = RelTimeIndicatorConverter.convert( - resultTable.getRelNode, - streamUtil.tableEnv.getRelBuilder.getRexBuilder) - val joinNode = relNode.getInput(0).asInstanceOf[LogicalJoin] - val (windowBounds, _) = - WindowJoinUtil.extractWindowBoundsFromPredicate( - joinNode.getCondition, - 4, - joinNode.getRowType, - joinNode.getCluster.getRexBuilder, - streamUtil.tableEnv.getConfig) - - val timeTypeStr = - if (windowBounds.get.isEventTime) "rowtime" - else "proctime" - assertEquals(expLeftSize, windowBounds.get.leftLowerBound) - assertEquals(expRightSize, windowBounds.get.leftUpperBound) - assertEquals(expTimeType, timeTypeStr) - } - @Test def testLeftOuterJoinEquiPred(): Unit = { val util = streamTestUtil() @@ -1009,6 +980,35 @@ class JoinTest extends TableTestBase { util.verifyTable(result, expected) } + private def verifyTimeBoundary( + timeSql: String, + expLeftSize: Long, + expRightSize: Long, + expTimeType: String): Unit = { + val query = + "SELECT t1.a, t2.b FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " + timeSql + + val resultTable = streamUtil.tableEnv.sqlQuery(query) + val relNode = RelTimeIndicatorConverter.convert( + resultTable.getRelNode, + streamUtil.tableEnv.getRelBuilder.getRexBuilder) + val joinNode = relNode.getInput(0).asInstanceOf[LogicalJoin] + val (windowBounds, _) = + WindowJoinUtil.extractWindowBoundsFromPredicate( + joinNode.getCondition, + 4, + joinNode.getRowType, + joinNode.getCluster.getRexBuilder, + streamUtil.tableEnv.getConfig) + + val timeTypeStr = + if (windowBounds.get.isEventTime) "rowtime" + else "proctime" + assertEquals(expLeftSize, windowBounds.get.leftLowerBound) + assertEquals(expRightSize, windowBounds.get.leftUpperBound) + assertEquals(expTimeType, timeTypeStr) + } + private def verifyRemainConditionConvert( query: String, expectCondStr: String): Unit = { From 75589e155b24405aada2eedabef74f794d9d5be2 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 16 Oct 2018 14:49:00 +0200 Subject: [PATCH 012/359] [hotfix] [table] Refactor SqlToConverter configuration This closes #6857. --- .../flink/table/api/TableEnvironment.scala | 36 +++++++++---------- .../table/calcite/FlinkPlannerImpl.scala | 4 +-- .../utils/ExpressionTestBase.scala | 3 +- .../match/PatternTranslatorTestBase.scala | 3 +- 4 files changed, 20 insertions(+), 26 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index 26f9e50fedd01..e28a471681d2c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -91,6 +91,7 @@ abstract class TableEnvironment(val config: TableConfig) { .costFactory(new DataSetCostFactory) .typeSystem(new FlinkTypeSystem) .operatorTable(getSqlOperatorTable) + .sqlToRelConverterConfig(getSqlToRelConverterConfig) // set the executor to evaluate constant expressions .executor(new ExpressionReducer(config)) .build @@ -109,15 +110,6 @@ abstract class TableEnvironment(val config: TableConfig) { // registered external catalog names -> catalog private val externalCatalogs = new mutable.HashMap[String, ExternalCatalog] - // configuration for SqlToRelConverter - private[flink] lazy val sqlToRelConverterConfig: SqlToRelConverter.Config = { - val calciteConfig = config.getCalciteConfig - calciteConfig.getSqlToRelConverterConfig match { - case Some(c) => c - case None => getSqlToRelConverterConfig - } - } - /** Returns the table config to define the runtime behavior of the Table API. */ def getConfig: TableConfig = config @@ -132,11 +124,18 @@ abstract class TableEnvironment(val config: TableConfig) { * Returns the SqlToRelConverter config. */ protected def getSqlToRelConverterConfig: SqlToRelConverter.Config = { - SqlToRelConverter.configBuilder() - .withTrimUnusedFields(false) - .withConvertTableAccess(false) - .withInSubQueryThreshold(Integer.MAX_VALUE) - .build() + val calciteConfig = config.getCalciteConfig + calciteConfig.getSqlToRelConverterConfig match { + + case None => + SqlToRelConverter.configBuilder() + .withTrimUnusedFields(false) + .withConvertTableAccess(false) + .withInSubQueryThreshold(Integer.MAX_VALUE) + .build() + + case Some(c) => c + } } /** @@ -717,8 +716,7 @@ abstract class TableEnvironment(val config: TableConfig) { val planner = new FlinkPlannerImpl( getFrameworkConfig, getPlanner, - getTypeFactory, - sqlToRelConverterConfig) + getTypeFactory) planner.getCompletionHints(statement, position) } @@ -740,8 +738,7 @@ abstract class TableEnvironment(val config: TableConfig) { * @return The result of the query as Table */ def sqlQuery(query: String): Table = { - val planner = new FlinkPlannerImpl( - getFrameworkConfig, getPlanner, getTypeFactory, sqlToRelConverterConfig) + val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner, getTypeFactory) // parse the sql query val parsed = planner.parse(query) if (null != parsed && parsed.getKind.belongsTo(SqlKind.QUERY)) { @@ -801,8 +798,7 @@ abstract class TableEnvironment(val config: TableConfig) { * @param config The [[QueryConfig]] to use. */ def sqlUpdate(stmt: String, config: QueryConfig): Unit = { - val planner = new FlinkPlannerImpl( - getFrameworkConfig, getPlanner, getTypeFactory, sqlToRelConverterConfig) + val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner, getTypeFactory) // parse the sql query val parsed = planner.parse(stmt) parsed match { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala index ca300539f7b33..400279d8c7268 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala @@ -48,8 +48,7 @@ import scala.collection.JavaConversions._ class FlinkPlannerImpl( config: FrameworkConfig, planner: RelOptPlanner, - typeFactory: FlinkTypeFactory, - sqlToRelConverterConfig: SqlToRelConverter.Config) { + typeFactory: FlinkTypeFactory) { val operatorTable: SqlOperatorTable = config.getOperatorTable /** Holds the trait definitions to be registered with planner. May be null. */ @@ -57,6 +56,7 @@ class FlinkPlannerImpl( val parserConfig: SqlParser.Config = config.getParserConfig val convertletTable: SqlRexConvertletTable = config.getConvertletTable val defaultSchema: SchemaPlus = config.getDefaultSchema + val sqlToRelConverterConfig: SqlToRelConverter.Config = config.getSqlToRelConverterConfig var validator: FlinkCalciteSqlValidator = _ var root: RelRoot = _ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala index b8adfa78efbc7..f9220e68f0f3f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala @@ -67,8 +67,7 @@ abstract class ExpressionTestBase { private val planner = new FlinkPlannerImpl( context._2.getFrameworkConfig, context._2.getPlanner, - context._2.getTypeFactory, - context._2.sqlToRelConverterConfig) + context._2.getTypeFactory) private val logicalOptProgram = Programs.ofRules(FlinkRuleSets.LOGICAL_OPT_RULES) private val dataSetOptProgram = Programs.ofRules(FlinkRuleSets.DATASET_OPT_RULES) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala index 55bdb5245c2be..883ce0adf8102 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala @@ -49,8 +49,7 @@ abstract class PatternTranslatorTestBase extends TestLogger{ private val planner = new FlinkPlannerImpl( context._2.getFrameworkConfig, context._2.getPlanner, - context._2.getTypeFactory, - context._2.sqlToRelConverterConfig) + context._2.getTypeFactory) private def prepareContext(typeInfo: TypeInformation[Row]) : (RelBuilder, StreamTableEnvironment, StreamExecutionEnvironment) = { From 82c52670a9d13b629b6af387fe0a6ca9b7bd5966 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 23 Oct 2018 18:47:02 +0200 Subject: [PATCH 013/359] [FLINK-10655][rpc] fix RemoteRpcInvocation not overwriting ObjectInputStream's ClassNotFoundException --- .../base/EnumSerializerUpgradeTest.java | 37 ++---------- .../flink/testutils/ClassLoaderUtils.java | 59 +++++++++++++++++++ .../rpc/messages/RemoteRpcInvocation.java | 42 ++++++++++--- .../runtime/classloading/ClassLoaderTest.java | 59 ++++++++++++++++++- 4 files changed, 155 insertions(+), 42 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java index e906f62f65bf6..fb11945305721 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java @@ -23,21 +23,17 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.testutils.ClassLoaderUtils; import org.apache.flink.util.TestLogger; + import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.tools.JavaCompiler; -import javax.tools.ToolProvider; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileWriter; import java.io.IOException; -import java.net.URL; -import java.net.URLClassLoader; public class EnumSerializerUpgradeTest extends TestLogger { @@ -87,7 +83,7 @@ public void checkDifferentFieldOrder() throws Exception { private static TypeSerializerSchemaCompatibility checkCompatibility(String enumSourceA, String enumSourceB) throws IOException, ClassNotFoundException { - ClassLoader classLoader = compileAndLoadEnum( + ClassLoader classLoader = ClassLoaderUtils.compileAndLoadJava( temporaryFolder.newFolder(), ENUM_NAME + ".java", enumSourceA); EnumSerializer enumSerializer = new EnumSerializer(classLoader.loadClass(ENUM_NAME)); @@ -103,7 +99,7 @@ private static TypeSerializerSchemaCompatibility checkCompatibility(String enumS snapshotBytes = outBuffer.toByteArray(); } - ClassLoader classLoader2 = compileAndLoadEnum( + ClassLoader classLoader2 = ClassLoaderUtils.compileAndLoadJava( temporaryFolder.newFolder(), ENUM_NAME + ".java", enumSourceB); TypeSerializerSnapshot restoredSnapshot; @@ -118,29 +114,4 @@ private static TypeSerializerSchemaCompatibility checkCompatibility(String enumS EnumSerializer enumSerializer2 = new EnumSerializer(classLoader2.loadClass(ENUM_NAME)); return restoredSnapshot.resolveSchemaCompatibility(enumSerializer2); } - - private static ClassLoader compileAndLoadEnum(File root, String filename, String source) throws IOException { - File file = writeSourceFile(root, filename, source); - - compileClass(file); - - return new URLClassLoader( - new URL[]{root.toURI().toURL()}, - Thread.currentThread().getContextClassLoader()); - } - - private static File writeSourceFile(File root, String filename, String source) throws IOException { - File file = new File(root, filename); - FileWriter fileWriter = new FileWriter(file); - - fileWriter.write(source); - fileWriter.close(); - - return file; - } - - private static int compileClass(File sourceFile) { - JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); - return compiler.run(null, null, null, "-proc:none", sourceFile.getPath()); - } } diff --git a/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java b/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java new file mode 100644 index 0000000000000..0688c1df15638 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java @@ -0,0 +1,59 @@ +/* + * 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.testutils; + +import javax.tools.JavaCompiler; +import javax.tools.ToolProvider; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; + +/** + * Utilities to create class loaders. + */ +public class ClassLoaderUtils { + public static URLClassLoader compileAndLoadJava(File root, String filename, String source) throws + IOException { + File file = writeSourceFile(root, filename, source); + + compileClass(file); + + return new URLClassLoader( + new URL[]{root.toURI().toURL()}, + Thread.currentThread().getContextClassLoader()); + } + + private static File writeSourceFile(File root, String filename, String source) throws IOException { + File file = new File(root, filename); + FileWriter fileWriter = new FileWriter(file); + + fileWriter.write(source); + fileWriter.close(); + + return file; + } + + private static int compileClass(File sourceFile) { + JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); + return compiler.run(null, null, null, "-proc:none", sourceFile.getPath()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java index 7b9fb88767043..486816de8e80d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java @@ -203,12 +203,18 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound try { parameterTypes[i] = (Class) ois.readObject(); } catch (IOException e) { + StringBuilder incompleteMethod = getIncompleteMethodString(i, 0); throw new IOException("Could not deserialize " + i + "th parameter type of method " + - methodName + '.', e); + incompleteMethod + '.', e); } catch (ClassNotFoundException e) { - throw new ClassNotFoundException("Could not deserialize " + i + "th " + - "parameter type of method " + methodName + ". This indicates that the parameter " + - "type is not part of the system class loader.", e); + // note: wrapping this CNFE into another CNFE does not overwrite the Exception + // stored in the ObjectInputStream (see ObjectInputStream#readSerialData) + // -> add a suppressed exception that adds a more specific message + StringBuilder incompleteMethod = getIncompleteMethodString(i, 0); + e.addSuppressed(new ClassNotFoundException("Could not deserialize " + i + "th " + + "parameter type of method " + incompleteMethod + ". This indicates that the parameter " + + "type is not part of the system class loader.")); + throw e; } } @@ -221,17 +227,37 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound try { args[i] = ois.readObject(); } catch (IOException e) { + StringBuilder incompleteMethod = getIncompleteMethodString(length, i); throw new IOException("Could not deserialize " + i + "th argument of method " + - methodName + '.', e); + incompleteMethod + '.', e); } catch (ClassNotFoundException e) { - throw new ClassNotFoundException("Could not deserialize " + i + "th " + - "argument of method " + methodName + ". This indicates that the argument " + - "type is not part of the system class loader.", e); + // note: wrapping this CNFE into another CNFE does not overwrite the Exception + // stored in the ObjectInputStream (see ObjectInputStream#readSerialData) + // -> add a suppressed exception that adds a more specific message + StringBuilder incompleteMethod = getIncompleteMethodString(length, i); + e.addSuppressed(new ClassNotFoundException("Could not deserialize " + i + "th " + + "argument of method " + incompleteMethod + ". This indicates that the argument " + + "type is not part of the system class loader.")); + throw e; } } } else { args = null; } } + + private StringBuilder getIncompleteMethodString(int lastMethodTypeIdx, int lastArgumentIdx) { + StringBuilder incompleteMethod = new StringBuilder(); + incompleteMethod.append(methodName).append('('); + for (int i = 0; i < lastMethodTypeIdx; ++i) { + incompleteMethod.append(parameterTypes[i].getCanonicalName()); + if (i < lastArgumentIdx) { + incompleteMethod.append(": ").append(args[i]); + } + incompleteMethod.append(", "); + } + incompleteMethod.append("...)"); // some parameters could not be deserialized + return incompleteMethod; + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java index c02278c508887..7c664ce0b3206 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java @@ -19,21 +19,78 @@ package org.apache.flink.runtime.classloading; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation; +import org.apache.flink.testutils.ClassLoaderUtils; +import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; import java.net.URL; import java.net.URLClassLoader; +import static org.hamcrest.CoreMatchers.allOf; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.isA; +import static org.hamcrest.Matchers.hasItemInArray; +import static org.hamcrest.Matchers.hasProperty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; /** - * Tests for classloading and class loder utilities. + * Tests for classloading and class loader utilities. */ public class ClassLoaderTest extends TestLogger { + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testMessageDecodingWithUnavailableClass() throws Exception { + final ClassLoader systemClassLoader = getClass().getClassLoader(); + + final String className = "UserClass"; + final URLClassLoader userClassLoader = ClassLoaderUtils.compileAndLoadJava( + temporaryFolder.newFolder(), + className + ".java", + "import java.io.Serializable;\n" + + "public class " + className + " implements Serializable {}"); + + RemoteRpcInvocation method = new RemoteRpcInvocation( + "test", + new Class[] { + int.class, + Class.forName(className, false, userClassLoader)}, + new Object[] { + 1, + Class.forName(className, false, userClassLoader).newInstance()}); + + SerializedValue serializedMethod = new SerializedValue<>(method); + + expectedException.expect(ClassNotFoundException.class); + expectedException.expect( + allOf( + isA(ClassNotFoundException.class), + hasProperty("suppressed", + hasItemInArray( + allOf( + isA(ClassNotFoundException.class), + hasProperty("message", + containsString("Could not deserialize 1th parameter type of method test(int, ...)."))))))); + + RemoteRpcInvocation deserializedMethod = serializedMethod.deserializeValue(systemClassLoader); + deserializedMethod.getMethodName(); + + userClassLoader.close(); + } + @Test public void testParentFirstClassLoading() throws Exception { final ClassLoader parentClassLoader = getClass().getClassLoader(); From 48ef49cec69a739cd9446eb8add3483dc8c882a7 Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 25 Oct 2018 19:00:34 +0800 Subject: [PATCH 014/359] [FLINK-10600] Provide End-to-end test cases for modern Kafka connectors --- .../flink-connector-kafka/pom.xml | 4 + flink-dist/src/main/assemblies/bin.xml | 28 ++++ .../run-pre-commit-tests.sh | 1 + .../test-scripts/kafka-common.sh | 29 ++-- .../test-scripts/test_streaming_kafka.sh | 27 ++++ .../test-scripts/test_streaming_kafka010.sh | 93 +---------- .../test_streaming_kafka_common.sh | 117 ++++++++++++++ .../pom.xml | 88 +++++++++++ .../examples/kafka/Kafka010Example.java | 69 +++++++++ .../examples/kafka/Kafka010Example.scala | 2 +- .../pom.xml | 43 ++++++ .../kafka/base/CustomWatermarkExtractor.java | 52 +++++++ .../examples/kafka/base}/KafkaEvent.java | 2 +- .../kafka/base}/KafkaEventSchema.java | 2 +- .../examples/kafka/base/KafkaExampleUtil.java | 54 +++++++ .../kafka/base/RollingAdditionMapper.java | 53 +++++++ .../flink-examples-streaming-kafka/pom.xml | 89 +++++++++++ .../examples/kafka/KafkaExample.java | 72 +++++++++ .../flink-examples-streaming/pom.xml | 37 ----- .../examples/kafka/Kafka010Example.java | 146 ------------------ flink-examples/pom.xml | 3 + 21 files changed, 720 insertions(+), 291 deletions(-) create mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh create mode 100644 flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh create mode 100644 flink-examples/flink-examples-streaming-kafka-0.10/pom.xml create mode 100644 flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java rename flink-examples/{flink-examples-streaming => flink-examples-streaming-kafka-0.10}/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala (97%) create mode 100644 flink-examples/flink-examples-streaming-kafka-base/pom.xml create mode 100644 flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java rename flink-examples/{flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka => flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base}/KafkaEvent.java (97%) rename flink-examples/{flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka => flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base}/KafkaEventSchema.java (97%) create mode 100644 flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.java create mode 100644 flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.java create mode 100644 flink-examples/flink-examples-streaming-kafka/pom.xml create mode 100644 flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java delete mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index 109535aa4db65..10f738337201d 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -52,6 +52,10 @@ under the License. org.apache.kafka kafka_${scala.binary.version} + + org.apache.kafka + kafka-clients + diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 89228b39271f8..e1adab537d968 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -187,6 +187,34 @@ under the License. + + + ../flink-examples/flink-examples-streaming-kafka/target + examples/streaming + 0644 + + *.jar + + + flink-examples-streaming-kafka*.jar + original-*.jar + + + + + + ../flink-examples/flink-examples-streaming-kafka-0.10/target + examples/streaming + 0644 + + *.jar + + + flink-examples-streaming-kafka*.jar + original-*.jar + + + ../flink-libraries/flink-gelly-examples/target diff --git a/flink-end-to-end-tests/run-pre-commit-tests.sh b/flink-end-to-end-tests/run-pre-commit-tests.sh index 15b9b152f7740..3e3494a491a49 100755 --- a/flink-end-to-end-tests/run-pre-commit-tests.sh +++ b/flink-end-to-end-tests/run-pre-commit-tests.sh @@ -54,6 +54,7 @@ run_test "Batch Python Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/ run_test "Streaming Python Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_python_wordcount.sh" run_test "Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_wordcount.sh" run_test "Kafka end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka010.sh" +run_test "Modern Kafka end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka.sh" run_test "class loading end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_classloader.sh" run_test "Shaded Hadoop S3A end-to-end test" "$END_TO_END_DIR/test-scripts/test_shaded_hadoop_s3a.sh" run_test "Shaded Presto S3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_shaded_presto_s3.sh" diff --git a/flink-end-to-end-tests/test-scripts/kafka-common.sh b/flink-end-to-end-tests/test-scripts/kafka-common.sh index 2dc58f7305a19..dedfe5208b1ae 100644 --- a/flink-end-to-end-tests/test-scripts/kafka-common.sh +++ b/flink-end-to-end-tests/test-scripts/kafka-common.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +set -e +set -u set -o pipefail if [[ -z $TEST_DATA_DIR ]]; then @@ -24,15 +26,19 @@ if [[ -z $TEST_DATA_DIR ]]; then exit 1 fi -KAFKA_DIR=$TEST_DATA_DIR/kafka_2.11-0.10.2.0 -CONFLUENT_DIR=$TEST_DATA_DIR/confluent-3.2.0 +KAFKA_VERSION="$1" +CONFLUENT_VERSION="$2" +CONFLUENT_MAJOR_VERSION="$3" + +KAFKA_DIR=$TEST_DATA_DIR/kafka_2.11-$KAFKA_VERSION +CONFLUENT_DIR=$TEST_DATA_DIR/confluent-$CONFLUENT_VERSION SCHEMA_REGISTRY_PORT=8082 SCHEMA_REGISTRY_URL=http://localhost:${SCHEMA_REGISTRY_PORT} function setup_kafka_dist { # download Kafka mkdir -p $TEST_DATA_DIR - KAFKA_URL="https://archive.apache.org/dist/kafka/0.10.2.0/kafka_2.11-0.10.2.0.tgz" + KAFKA_URL="https://archive.apache.org/dist/kafka/$KAFKA_VERSION/kafka_2.11-$KAFKA_VERSION.tgz" echo "Downloading Kafka from $KAFKA_URL" curl "$KAFKA_URL" > $TEST_DATA_DIR/kafka.tgz @@ -46,7 +52,7 @@ function setup_kafka_dist { function setup_confluent_dist { # download confluent mkdir -p $TEST_DATA_DIR - CONFLUENT_URL="http://packages.confluent.io/archive/3.2/confluent-oss-3.2.0-2.11.tar.gz" + CONFLUENT_URL="http://packages.confluent.io/archive/$CONFLUENT_MAJOR_VERSION/confluent-oss-$CONFLUENT_VERSION-2.11.tar.gz" echo "Downloading confluent from $CONFLUENT_URL" curl "$CONFLUENT_URL" > $TEST_DATA_DIR/confluent.tgz @@ -76,13 +82,15 @@ function stop_kafka_cluster { $KAFKA_DIR/bin/kafka-server-stop.sh $KAFKA_DIR/bin/zookeeper-server-stop.sh - PIDS=$(jps -vl | grep -i 'kafka\.Kafka' | grep java | grep -v grep | awk '{print $1}') + # Terminate Kafka process if it still exists + PIDS=$(jps -vl | grep -i 'kafka\.Kafka' | grep java | grep -v grep | awk '{print $1}'|| echo "") if [ ! -z "$PIDS" ]; then kill -s TERM $PIDS fi - PIDS=$(jps -vl | grep java | grep -i QuorumPeerMain | grep -v grep | awk '{print $1}') + # Terminate QuorumPeerMain process if it still exists + PIDS=$(jps -vl | grep java | grep -i QuorumPeerMain | grep -v grep | awk '{print $1}'|| echo "") if [ ! -z "$PIDS" ]; then kill -s TERM $PIDS @@ -120,14 +128,7 @@ function get_partition_end_offset { local topic=$1 local partition=$2 - # first, use the console consumer to produce a dummy consumer group - read_messages_from_kafka 0 $topic dummy-consumer - - # then use the consumer offset utility to get the LOG_END_OFFSET value for the specified partition - $KAFKA_DIR/bin/kafka-consumer-groups.sh --describe --group dummy-consumer --bootstrap-server localhost:9092 2> /dev/null \ - | grep "$topic \+$partition" \ - | tr -s " " \ - | cut -d " " -f 4 + $KAFKA_DIR/bin/kafka-run-class.sh kafka.tools.GetOffsetShell --broker-list localhost:9092 --topic $topic --partitions $partition --time -1 | cut -d ":" -f 3 } function start_confluent_schema_registry { diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh new file mode 100755 index 0000000000000..c5cdfde3dc0e5 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +set -e +set -u +set -o pipefail + +source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/kafka-common.sh 2.0.0 5.0.0 5.0 + +source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/KafkaExample.jar diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh index c9cc19db71d55..ecd651448a975 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh @@ -18,96 +18,7 @@ ################################################################################ source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh +source "$(dirname "$0")"/kafka-common.sh 0.10.2.0 3.2.0 3.2 -setup_kafka_dist -start_kafka_cluster +source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/Kafka010Example.jar -# modify configuration to have enough slots -cp $FLINK_DIR/conf/flink-conf.yaml $FLINK_DIR/conf/flink-conf.yaml.bak -sed -i -e "s/taskmanager.numberOfTaskSlots: 1/taskmanager.numberOfTaskSlots: 3/" $FLINK_DIR/conf/flink-conf.yaml - -start_cluster - -function test_cleanup { - # don't call ourselves again for another signal interruption - trap "exit -1" INT - # don't call ourselves again for normal exit - trap "" EXIT - - stop_kafka_cluster - - # revert our modifications to the Flink distribution - mv -f $FLINK_DIR/conf/flink-conf.yaml.bak $FLINK_DIR/conf/flink-conf.yaml -} -trap test_cleanup INT -trap test_cleanup EXIT - -# create the required topics -create_kafka_topic 1 1 test-input -create_kafka_topic 1 1 test-output - -# run the Flink job (detached mode) -$FLINK_DIR/bin/flink run -d $FLINK_DIR/examples/streaming/Kafka010Example.jar \ - --input-topic test-input --output-topic test-output \ - --prefix=PREFIX \ - --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer --auto.offset.reset earliest \ - --flink.partition-discovery.interval-millis 1000 - -function verify_output { - local expected=$(printf $1) - - if [[ "$2" != "$expected" ]]; then - echo "Output from Flink program does not match expected output." - echo -e "EXPECTED FOR KEY: --$expected--" - echo -e "ACTUAL: --$2--" - exit 1 - fi -} - -echo "Sending messages to Kafka topic [test-input] ..." -# send some data to Kafka -send_messages_to_kafka "elephant,5,45218\nsquirrel,12,46213\nbee,3,51348\nsquirrel,22,52444\nbee,10,53412\nelephant,9,54867" test-input - -echo "Verifying messages from Kafka topic [test-output] ..." - -KEY_1_MSGS=$(read_messages_from_kafka 6 test-output elephant_consumer | grep elephant) -KEY_2_MSGS=$(read_messages_from_kafka 6 test-output squirrel_consumer | grep squirrel) -KEY_3_MSGS=$(read_messages_from_kafka 6 test-output bee_consumer | grep bee) - -# check all keys; make sure we have actual newlines in the string, not "\n" -verify_output "elephant,5,45218\nelephant,14,54867" "$KEY_1_MSGS" -verify_output "squirrel,12,46213\nsquirrel,34,52444" "$KEY_2_MSGS" -verify_output "bee,3,51348\nbee,13,53412" "$KEY_3_MSGS" - -# now, we add a new partition to the topic -echo "Repartitioning Kafka topic [test-input] ..." -modify_num_partitions test-input 2 - -if (( $(get_num_partitions test-input) != 2 )); then - echo "Failed adding a partition to test-input topic." - exit 1 -fi - -# send some more messages to Kafka -echo "Sending more messages to Kafka topic [test-input] ..." -send_messages_to_kafka "elephant,13,64213\ngiraffe,9,65555\nbee,5,65647\nsquirrel,18,66413" test-input - -# verify that our assumption that the new partition actually has written messages is correct -if (( $(get_partition_end_offset test-input 1) == 0 )); then - echo "The newly created partition does not have any new messages, and therefore partition discovery cannot be verified." - exit 1 -fi - -# all new messages should have been consumed, and has produced correct output -echo "Verifying messages from Kafka topic [test-output] ..." - -KEY_1_MSGS=$(read_messages_from_kafka 4 test-output elephant_consumer | grep elephant) -KEY_2_MSGS=$(read_messages_from_kafka 4 test-output squirrel_consumer | grep squirrel) -KEY_3_MSGS=$(read_messages_from_kafka 4 test-output bee_consumer | grep bee) -KEY_4_MSGS=$(read_messages_from_kafka 10 test-output giraffe_consumer | grep giraffe) - -verify_output "elephant,27,64213" "$KEY_1_MSGS" -verify_output "squirrel,52,66413" "$KEY_2_MSGS" -verify_output "bee,18,65647" "$KEY_3_MSGS" -verify_output "giraffe,9,65555" "$KEY_4_MSGS" diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh new file mode 100644 index 0000000000000..ff3adc158c7fd --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh @@ -0,0 +1,117 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +set -e +set -u +set -o pipefail + +KAFKA_EXAMPLE_JAR="$1" + +setup_kafka_dist +start_kafka_cluster + +# modify configuration to have enough slots +cp $FLINK_DIR/conf/flink-conf.yaml $FLINK_DIR/conf/flink-conf.yaml.bak +sed -i -e "s/taskmanager.numberOfTaskSlots: 1/taskmanager.numberOfTaskSlots: 3/" $FLINK_DIR/conf/flink-conf.yaml + +start_cluster + +function test_cleanup { + # don't call ourselves again for another signal interruption + trap "exit -1" INT + # don't call ourselves again for normal exit + trap "" EXIT + + stop_kafka_cluster + + # revert our modifications to the Flink distribution + mv -f $FLINK_DIR/conf/flink-conf.yaml.bak $FLINK_DIR/conf/flink-conf.yaml +} +trap test_cleanup INT +trap test_cleanup EXIT + +# create the required topics +create_kafka_topic 1 1 test-input +create_kafka_topic 1 1 test-output + +# run the Flink job (detached mode) +$FLINK_DIR/bin/flink run -d $KAFKA_EXAMPLE_JAR \ + --input-topic test-input --output-topic test-output \ + --prefix=PREFIX \ + --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer --auto.offset.reset earliest \ + --transaction.timeout.ms 900000 \ + --flink.partition-discovery.interval-millis 1000 + +function verify_output { + local expected=$(printf $1) + + if [[ "$2" != "$expected" ]]; then + echo "Output from Flink program does not match expected output." + echo -e "EXPECTED FOR KEY: --$expected--" + echo -e "ACTUAL: --$2--" + exit 1 + fi +} + +echo "Sending messages to Kafka topic [test-input] ..." +# send some data to Kafka +send_messages_to_kafka "elephant,5,45218\nsquirrel,12,46213\nbee,3,51348\nsquirrel,22,52444\nbee,10,53412\nelephant,9,54867" test-input + +echo "Verifying messages from Kafka topic [test-output] ..." + +KEY_1_MSGS=$(read_messages_from_kafka 6 test-output elephant_consumer | grep elephant) +KEY_2_MSGS=$(read_messages_from_kafka 6 test-output squirrel_consumer | grep squirrel) +KEY_3_MSGS=$(read_messages_from_kafka 6 test-output bee_consumer | grep bee) + +# check all keys; make sure we have actual newlines in the string, not "\n" +verify_output "elephant,5,45218\nelephant,14,54867" "$KEY_1_MSGS" +verify_output "squirrel,12,46213\nsquirrel,34,52444" "$KEY_2_MSGS" +verify_output "bee,3,51348\nbee,13,53412" "$KEY_3_MSGS" + +# now, we add a new partition to the topic +echo "Repartitioning Kafka topic [test-input] ..." +modify_num_partitions test-input 2 + +if (( $(get_num_partitions test-input) != 2 )); then + echo "Failed adding a partition to test-input topic." + exit 1 +fi + +# send some more messages to Kafka +echo "Sending more messages to Kafka topic [test-input] ..." +send_messages_to_kafka "elephant,13,64213\ngiraffe,9,65555\nbee,5,65647\nsquirrel,18,66413" test-input + +# verify that our assumption that the new partition actually has written messages is correct +if (( $(get_partition_end_offset test-input 1) == 0 )); then + echo "The newly created partition does not have any new messages, and therefore partition discovery cannot be verified." + exit 1 +fi + +# all new messages should have been consumed, and has produced correct output +echo "Verifying messages from Kafka topic [test-output] ..." + +KEY_1_MSGS=$(read_messages_from_kafka 4 test-output elephant_consumer | grep elephant) +KEY_2_MSGS=$(read_messages_from_kafka 4 test-output squirrel_consumer | grep squirrel) +KEY_3_MSGS=$(read_messages_from_kafka 4 test-output bee_consumer | grep bee) +KEY_4_MSGS=$(read_messages_from_kafka 10 test-output giraffe_consumer | grep giraffe) + +verify_output "elephant,27,64213" "$KEY_1_MSGS" +verify_output "squirrel,52,66413" "$KEY_2_MSGS" +verify_output "bee,18,65647" "$KEY_3_MSGS" +verify_output "giraffe,9,65555" "$KEY_4_MSGS" diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml b/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml new file mode 100644 index 0000000000000..ced3919da3e57 --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml @@ -0,0 +1,88 @@ + + + + + flink-examples + org.apache.flink + 1.8-SNAPSHOT + + 4.0.0 + + flink-examples-streaming-kafka-0.10 + + + + org.apache.flink + flink-examples-streaming-kafka-base + ${project.version} + + + + org.apache.flink + flink-connector-kafka-0.10_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-java + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + fat-jar-kafka-example + package + + shade + + + false + false + false + + + org.apache.flink.streaming.examples.kafka.Kafka010Example + + + Kafka010Example + + + + + + + + diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java b/flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java new file mode 100644 index 0000000000000..2df1f5d84963a --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.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.streaming.examples.kafka; + +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.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.examples.kafka.base.CustomWatermarkExtractor; +import org.apache.flink.streaming.examples.kafka.base.KafkaEvent; +import org.apache.flink.streaming.examples.kafka.base.KafkaEventSchema; +import org.apache.flink.streaming.examples.kafka.base.KafkaExampleUtil; +import org.apache.flink.streaming.examples.kafka.base.RollingAdditionMapper; + +/** + * A simple example that shows how to read from and write to Kafka. This will read String messages + * from the input topic, parse them into a POJO type {@link KafkaEvent}, group by some key, and finally + * perform a rolling addition on each key for which the results are written back to another topic. + * + *

This example also demonstrates using a watermark assigner to generate per-partition + * watermarks directly in the Flink Kafka consumer. For demonstration purposes, it is assumed that + * the String messages are of formatted as a (word,frequency,timestamp) tuple. + * + *

Example usage: + * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer + */ +public class Kafka010Example { + + public static void main(String[] args) throws Exception { + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + StreamExecutionEnvironment env = KafkaExampleUtil.prepareExecutionEnv(parameterTool); + + DataStream input = env + .addSource( + new FlinkKafkaConsumer010<>( + parameterTool.getRequired("input-topic"), + new KafkaEventSchema(), + parameterTool.getProperties()) + .assignTimestampsAndWatermarks(new CustomWatermarkExtractor())) + .keyBy("word") + .map(new RollingAdditionMapper()); + + input.addSink( + new FlinkKafkaProducer010<>( + parameterTool.getRequired("output-topic"), + new KafkaEventSchema(), + parameterTool.getProperties())); + + env.execute("Kafka 0.10 Example"); + } + +} diff --git a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala b/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala similarity index 97% rename from flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala rename to flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala index 9f4fdc4c29489..c2ea5617a685f 100644 --- a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala +++ b/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala @@ -21,7 +21,7 @@ package org.apache.flink.streaming.scala.examples.kafka import org.apache.flink.api.common.restartstrategy.RestartStrategies import org.apache.flink.api.common.serialization.SimpleStringSchema import org.apache.flink.api.java.utils.ParameterTool -import org.apache.flink.streaming.api.scala._ +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.connectors.kafka.{FlinkKafkaConsumer010, FlinkKafkaProducer010} /** diff --git a/flink-examples/flink-examples-streaming-kafka-base/pom.xml b/flink-examples/flink-examples-streaming-kafka-base/pom.xml new file mode 100644 index 0000000000000..3f389f843a1a2 --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-base/pom.xml @@ -0,0 +1,43 @@ + + + + + flink-examples + org.apache.flink + 1.8-SNAPSHOT + + 4.0.0 + + flink-examples-streaming-kafka-base + flink-examples-streaming-kafka-base + + jar + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + + diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java new file mode 100644 index 0000000000000..51de582dc0f5c --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java @@ -0,0 +1,52 @@ +/* + * 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.examples.kafka.base; + +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.watermark.Watermark; + +import javax.annotation.Nullable; + +/** + * A custom {@link AssignerWithPeriodicWatermarks}, that simply assumes that the input stream + * records are strictly ascending. + * + *

Flink also ships some built-in convenience assigners, such as the + * {@link BoundedOutOfOrdernessTimestampExtractor} and {@link AscendingTimestampExtractor} + */ +public class CustomWatermarkExtractor implements AssignerWithPeriodicWatermarks { + + private static final long serialVersionUID = -742759155861320823L; + + private long currentTimestamp = Long.MIN_VALUE; + + @Override + public long extractTimestamp(KafkaEvent event, long previousElementTimestamp) { + // the inputs are assumed to be of format (message,timestamp) + this.currentTimestamp = event.getTimestamp(); + return event.getTimestamp(); + } + + @Nullable + @Override + public Watermark getCurrentWatermark() { + return new Watermark(currentTimestamp == Long.MIN_VALUE ? Long.MIN_VALUE : currentTimestamp - 1); + } +} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaEvent.java b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEvent.java similarity index 97% rename from flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaEvent.java rename to flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEvent.java index a144fc38257a9..7a8f84f8ca8e4 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaEvent.java +++ b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEvent.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka; +package org.apache.flink.streaming.examples.kafka.base; /** * The event type used in the {@link Kafka010Example}. diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaEventSchema.java b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEventSchema.java similarity index 97% rename from flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaEventSchema.java rename to flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEventSchema.java index 5b8e17dadca88..ea9c12b605651 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaEventSchema.java +++ b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEventSchema.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka; +package org.apache.flink.streaming.examples.kafka.base; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.java b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.java new file mode 100644 index 0000000000000..447dec246484b --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.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.examples.kafka.base; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +/** + * The util class for kafka example. + */ +public class KafkaExampleUtil { + + public static StreamExecutionEnvironment prepareExecutionEnv(ParameterTool parameterTool) + throws Exception { + + if (parameterTool.getNumberOfParameters() < 5) { + System.out.println("Missing parameters!\n" + + "Usage: Kafka --input-topic --output-topic " + + "--bootstrap.servers " + + "--zookeeper.connect --group.id "); + throw new Exception("Missing parameters!\n" + + "Usage: Kafka --input-topic --output-topic " + + "--bootstrap.servers " + + "--zookeeper.connect --group.id "); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000)); + env.enableCheckpointing(5000); // create a checkpoint every 5 seconds + env.getConfig().setGlobalJobParameters(parameterTool); // make parameters available in the web interface + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + + return env; + } + +} diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.java b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.java new file mode 100644 index 0000000000000..e71f86cefba64 --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.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.examples.kafka.base; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.configuration.Configuration; + +/** + * A {@link RichMapFunction} that continuously outputs the current total frequency count of a key. + * The current total count is keyed state managed by Flink. + */ +public class RollingAdditionMapper extends RichMapFunction { + + private static final long serialVersionUID = 1180234853172462378L; + + private transient ValueState currentTotalCount; + + @Override + public KafkaEvent map(KafkaEvent event) throws Exception { + Integer totalCount = currentTotalCount.value(); + + if (totalCount == null) { + totalCount = 0; + } + totalCount += event.getFrequency(); + + currentTotalCount.update(totalCount); + + return new KafkaEvent(event.getWord(), totalCount, event.getTimestamp()); + } + + @Override + public void open(Configuration parameters) throws Exception { + currentTotalCount = getRuntimeContext().getState(new ValueStateDescriptor<>("currentTotalCount", Integer.class)); + } +} diff --git a/flink-examples/flink-examples-streaming-kafka/pom.xml b/flink-examples/flink-examples-streaming-kafka/pom.xml new file mode 100644 index 0000000000000..0434460e98187 --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka/pom.xml @@ -0,0 +1,89 @@ + + + + + flink-examples + org.apache.flink + 1.8-SNAPSHOT + + 4.0.0 + + flink-examples-streaming-kafka + flink-examples-streaming-kafka + + + + org.apache.flink + flink-examples-streaming-kafka-base + ${project.version} + + + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-java + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + fat-jar-kafka-example + package + + shade + + + false + false + false + + + org.apache.flink.streaming.examples.kafka.KafkaExample + + + KafkaExample + + + + + + + + diff --git a/flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java b/flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java new file mode 100644 index 0000000000000..27e73d18c0817 --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java @@ -0,0 +1,72 @@ +/* + * 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.examples.kafka; + +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.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; +import org.apache.flink.streaming.examples.kafka.base.CustomWatermarkExtractor; +import org.apache.flink.streaming.examples.kafka.base.KafkaEvent; +import org.apache.flink.streaming.examples.kafka.base.KafkaEventSchema; +import org.apache.flink.streaming.examples.kafka.base.KafkaExampleUtil; +import org.apache.flink.streaming.examples.kafka.base.RollingAdditionMapper; +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; + +/** + * A simple example that shows how to read from and write to modern Kafka. This will read String messages + * from the input topic, parse them into a POJO type {@link KafkaEvent}, group by some key, and finally + * perform a rolling addition on each key for which the results are written back to another topic. + * + *

This example also demonstrates using a watermark assigner to generate per-partition + * watermarks directly in the Flink Kafka consumer. For demonstration purposes, it is assumed that + * the String messages are of formatted as a (word,frequency,timestamp) tuple. + * + *

Example usage: + * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 + * --zookeeper.connect localhost:2181 --group.id myconsumer + */ +public class KafkaExample extends KafkaExampleUtil { + + public static void main(String[] args) throws Exception { + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + StreamExecutionEnvironment env = KafkaExampleUtil.prepareExecutionEnv(parameterTool); + + DataStream input = env + .addSource( + new FlinkKafkaConsumer<>( + parameterTool.getRequired("input-topic"), + new KafkaEventSchema(), + parameterTool.getProperties()) + .assignTimestampsAndWatermarks(new CustomWatermarkExtractor())) + .keyBy("word") + .map(new RollingAdditionMapper()); + + input.addSink( + new FlinkKafkaProducer<>( + parameterTool.getRequired("output-topic"), + new KeyedSerializationSchemaWrapper<>(new KafkaEventSchema()), + parameterTool.getProperties(), + FlinkKafkaProducer.Semantic.EXACTLY_ONCE)); + + env.execute("Modern Kafka Example"); + } + +} diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml index cbeef2a10cd04..3ee304bb87dde 100644 --- a/flink-examples/flink-examples-streaming/pom.xml +++ b/flink-examples/flink-examples-streaming/pom.xml @@ -531,43 +531,6 @@ under the License. - - fat-jar-kafka-010-example - package - - shade - - - false - false - false - - - org.apache.flink.streaming.examples.kafka.Kafka010Example - - - Kafka010Example - - - - * - - org/apache/flink/streaming/examples/kafka/** - org/apache/flink/streaming/** - org/apache/kafka/** - org/apache/curator/** - org/apache/zookeeper/** - org/apache/jute/** - org/I0Itec/** - jline/** - com/yammer/** - kafka/** - - - - - - fat-jar-twitter-example package diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java deleted file mode 100644 index 62bfd4fc6c5b4..0000000000000 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java +++ /dev/null @@ -1,146 +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.streaming.examples.kafka; - -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.state.ValueState; -import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; -import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; - -import javax.annotation.Nullable; - -/** - * A simple example that shows how to read from and write to Kafka. This will read String messages - * from the input topic, parse them into a POJO type {@link KafkaEvent}, group by some key, and finally - * perform a rolling addition on each key for which the results are written back to another topic. - * - *

This example also demonstrates using a watermark assigner to generate per-partition - * watermarks directly in the Flink Kafka consumer. For demonstration purposes, it is assumed that - * the String messages are of formatted as a (word,frequency,timestamp) tuple. - * - *

Example usage: - * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer - */ -public class Kafka010Example { - - public static void main(String[] args) throws Exception { - // parse input arguments - final ParameterTool parameterTool = ParameterTool.fromArgs(args); - - if (parameterTool.getNumberOfParameters() < 5) { - System.out.println("Missing parameters!\n" + - "Usage: Kafka --input-topic --output-topic " + - "--bootstrap.servers " + - "--zookeeper.connect --group.id "); - return; - } - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.getConfig().disableSysoutLogging(); - env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000)); - env.enableCheckpointing(5000); // create a checkpoint every 5 seconds - env.getConfig().setGlobalJobParameters(parameterTool); // make parameters available in the web interface - env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); - - DataStream input = env - .addSource( - new FlinkKafkaConsumer010<>( - parameterTool.getRequired("input-topic"), - new KafkaEventSchema(), - parameterTool.getProperties()) - .assignTimestampsAndWatermarks(new CustomWatermarkExtractor())) - .keyBy("word") - .map(new RollingAdditionMapper()); - - input.addSink( - new FlinkKafkaProducer010<>( - parameterTool.getRequired("output-topic"), - new KafkaEventSchema(), - parameterTool.getProperties())); - - env.execute("Kafka 0.10 Example"); - } - - /** - * A {@link RichMapFunction} that continuously outputs the current total frequency count of a key. - * The current total count is keyed state managed by Flink. - */ - private static class RollingAdditionMapper extends RichMapFunction { - - private static final long serialVersionUID = 1180234853172462378L; - - private transient ValueState currentTotalCount; - - @Override - public KafkaEvent map(KafkaEvent event) throws Exception { - Integer totalCount = currentTotalCount.value(); - - if (totalCount == null) { - totalCount = 0; - } - totalCount += event.getFrequency(); - - currentTotalCount.update(totalCount); - - return new KafkaEvent(event.getWord(), totalCount, event.getTimestamp()); - } - - @Override - public void open(Configuration parameters) throws Exception { - currentTotalCount = getRuntimeContext().getState(new ValueStateDescriptor<>("currentTotalCount", Integer.class)); - } - } - - /** - * A custom {@link AssignerWithPeriodicWatermarks}, that simply assumes that the input stream - * records are strictly ascending. - * - *

Flink also ships some built-in convenience assigners, such as the - * {@link BoundedOutOfOrdernessTimestampExtractor} and {@link AscendingTimestampExtractor} - */ - private static class CustomWatermarkExtractor implements AssignerWithPeriodicWatermarks { - - private static final long serialVersionUID = -742759155861320823L; - - private long currentTimestamp = Long.MIN_VALUE; - - @Override - public long extractTimestamp(KafkaEvent event, long previousElementTimestamp) { - // the inputs are assumed to be of format (message,timestamp) - this.currentTimestamp = event.getTimestamp(); - return event.getTimestamp(); - } - - @Nullable - @Override - public Watermark getCurrentWatermark() { - return new Watermark(currentTimestamp == Long.MIN_VALUE ? Long.MIN_VALUE : currentTimestamp - 1); - } - } -} diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 6a9b7e4379315..1c60ed63eccb4 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -35,6 +35,9 @@ under the License. flink-examples-batch flink-examples-streaming + flink-examples-streaming-kafka-base + flink-examples-streaming-kafka + flink-examples-streaming-kafka-0.10 flink-examples-table From f32996b590e8811b8a6b20e640e2b79e479c3b26 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 6 Nov 2018 12:14:36 +0100 Subject: [PATCH 015/359] [hotfix][kafka] Fix references to 1.8-SNAPSHOT --- flink-examples/flink-examples-streaming-kafka-0.10/pom.xml | 2 +- flink-examples/flink-examples-streaming-kafka-base/pom.xml | 2 +- flink-examples/flink-examples-streaming-kafka/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml b/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml index ced3919da3e57..d9550f6ef2f9f 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml @@ -23,7 +23,7 @@ under the License. flink-examples org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT 4.0.0 diff --git a/flink-examples/flink-examples-streaming-kafka-base/pom.xml b/flink-examples/flink-examples-streaming-kafka-base/pom.xml index 3f389f843a1a2..25f533c9ad279 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka-base/pom.xml @@ -23,7 +23,7 @@ under the License. flink-examples org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT 4.0.0 diff --git a/flink-examples/flink-examples-streaming-kafka/pom.xml b/flink-examples/flink-examples-streaming-kafka/pom.xml index 0434460e98187..fe58baa7ef30a 100644 --- a/flink-examples/flink-examples-streaming-kafka/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. flink-examples org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT 4.0.0 From 0ea67408744ca33542c8bc04ae4d1857a241fa3a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 6 Nov 2018 11:45:10 +0100 Subject: [PATCH 016/359] [FLINK-10633][prometheus] Add end-to-end test --- .../flink-end-to-end-tests-common/pom.xml | 63 +++++ .../flink/tests/util/AutoClosablePath.java | 44 ++++ .../flink/tests/util/AutoClosableProcess.java | 87 +++++++ .../flink/tests/util/CommandLineWrapper.java | 154 ++++++++++++ .../flink/tests/util/FlinkDistribution.java | 219 ++++++++++++++++++ .../pom.xml | 104 +++++++++ .../PrometheusReporterEndToEndITCase.java | 210 +++++++++++++++++ .../src/test/resources/log4j-test.properties | 27 +++ flink-end-to-end-tests/pom.xml | 18 ++ 9 files changed, 926 insertions(+) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java create mode 100644 flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java create mode 100644 flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml new file mode 100644 index 0000000000000..eece2d597bfad --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml @@ -0,0 +1,63 @@ + + + + + flink-end-to-end-tests + org.apache.flink + 1.7-SNAPSHOT + + + 4.0.0 + + flink-end-to-end-tests-common + 1.7-SNAPSHOT + + + + org.apache.flink + flink-core + ${project.version} + + + com.squareup.okhttp3 + okhttp + 3.7.0 + + + org.apache.flink + flink-shaded-jackson + + + junit + junit + ${junit.version} + compile + + + + org.apache.flink + flink-dist_${scala.binary.version} + ${project.version} + provided + + + + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.java new file mode 100644 index 0000000000000..0098889ead619 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosablePath.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.tests.util; + +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.nio.file.Path; + +/** + * Utility class to delete a given {@link Path} when exiting a try-with-resources statement. + */ +public final class AutoClosablePath implements AutoCloseable { + + private final Path path; + + public AutoClosablePath(final Path path) { + Preconditions.checkNotNull(path, "Path must not be null."); + Preconditions.checkArgument(path.isAbsolute(), "Path must be absolute."); + this.path = path; + } + + @Override + public void close() throws IOException { + FileUtils.deleteFileOrDirectory(path.toFile()); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java new file mode 100644 index 0000000000000..02359302e13f1 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.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.tests.util; + +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Utility class to terminate a given {@link Process} when exiting a try-with-resources statement. + */ +public class AutoClosableProcess implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(AutoClosableProcess.class); + + private final Process process; + + public AutoClosableProcess(final Process process) { + Preconditions.checkNotNull(process); + this.process = process; + } + + public static AutoClosableProcess runNonBlocking(String step, String... commands) throws IOException { + LOG.info("Step Started: " + step); + Process process = new ProcessBuilder() + .command(commands) + .inheritIO() + .start(); + return new AutoClosableProcess(process); + } + + public static void runBlocking(String step, String... commands) throws IOException { + runBlocking(step, Duration.ofSeconds(30), commands); + } + + public static void runBlocking(String step, Duration timeout, String... commands) throws IOException { + LOG.info("Step started: " + step); + Process process = new ProcessBuilder() + .command(commands) + .inheritIO() + .start(); + + try (AutoClosableProcess autoProcess = new AutoClosableProcess(process)) { + final boolean success = process.waitFor(timeout.toMillis(), TimeUnit.MILLISECONDS); + if (!success) { + throw new TimeoutException(); + } + } catch (TimeoutException | InterruptedException e) { + throw new RuntimeException(step + " failed due to timeout."); + } + LOG.info("Step complete: " + step); + } + + @Override + public void close() throws IOException { + if (process.isAlive()) { + process.destroy(); + try { + process.waitFor(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java new file mode 100644 index 0000000000000..50fd2f81f0256 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/CommandLineWrapper.java @@ -0,0 +1,154 @@ +/* + * 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.tests.util; + +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; + +/** + * Utility class for setting up command-line tool usages in a readable fashion. + */ +public enum CommandLineWrapper { + ; + + public static WGetBuilder wget(String url) { + return new WGetBuilder(url); + } + + /** + * Wrapper around wget used for downloading files. + */ + public static final class WGetBuilder { + + private final String url; + private Path targetDir; + + WGetBuilder(String url) { + this.url = url; + } + + public WGetBuilder targetDir(Path dir) { + this.targetDir = dir; + return this; + } + + public String[] build() { + final List commandsList = new ArrayList<>(5); + commandsList.add("wget"); + commandsList.add("-q"); // silent + //commandsList.add("--show-progress"); // enable progress bar + if (targetDir != null) { + commandsList.add("-P"); + commandsList.add(targetDir.toAbsolutePath().toString()); + } + commandsList.add(url); + return commandsList.toArray(new String[commandsList.size()]); + } + } + + public static SedBuilder sed(final String command, final Path file) { + return new SedBuilder(command, file); + } + + /** + * Wrapper around sed used for processing text. + */ + public static final class SedBuilder { + + private final String command; + private final Path file; + + private boolean inPlace = false; + + SedBuilder(final String command, final Path file) { + this.command = command; + this.file = file; + } + + public SedBuilder inPlace() { + inPlace = true; + return this; + } + + public String[] build() { + final List commandsList = new ArrayList<>(5); + commandsList.add("sed"); + if (inPlace) { + commandsList.add("-i"); + } + commandsList.add("-e"); + commandsList.add(command); + commandsList.add(file.toAbsolutePath().toString()); + return commandsList.toArray(new String[commandsList.size()]); + } + } + + public static TarBuilder tar(final Path file) { + return new TarBuilder(file); + } + + /** + * Wrapper around tar used for extracting .tar archives. + */ + public static final class TarBuilder { + + private final Path file; + private boolean zipped = false; + private boolean extract = false; + private Path targetDir; + + public TarBuilder(final Path file) { + this.file = file; + } + + public TarBuilder zipped() { + zipped = true; + return this; + } + + public TarBuilder extract() { + extract = true; + return this; + } + + public TarBuilder targetDir(final Path dir) { + targetDir = dir; + return this; + } + + public String[] build() { + final List commandsList = new ArrayList<>(4); + commandsList.add("tar"); + if (zipped) { + commandsList.add("-z"); + } + if (extract) { + commandsList.add("-x"); + } + if (targetDir != null) { + commandsList.add("--directory"); + commandsList.add(targetDir.toAbsolutePath().toString()); + } + commandsList.add("-f"); + commandsList.add(file.toAbsolutePath().toString()); + return commandsList.toArray(new String[commandsList.size()]); + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java new file mode 100644 index 0000000000000..7031d88c89013 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -0,0 +1,219 @@ +/* + * 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.tests.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.util.ExceptionUtils; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; +import org.junit.Assert; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedReader; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * A wrapper around a Flink distribution. + */ +public final class FlinkDistribution extends ExternalResource { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml"); + private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak"); + + private final List filesToDelete = new ArrayList<>(4); + + private final Path opt; + private final Path lib; + private final Path conf; + private final Path log; + private final Path bin; + + private Configuration defaultConfig; + + public FlinkDistribution() { + final String distDirProperty = System.getProperty("distDir"); + if (distDirProperty == null) { + Assert.fail("The distDir property was not set. You can set it when running maven via -DdistDir= ."); + } + final Path flinkDir = Paths.get(distDirProperty); + bin = flinkDir.resolve("bin"); + opt = flinkDir.resolve("opt"); + lib = flinkDir.resolve("lib"); + conf = flinkDir.resolve("conf"); + log = flinkDir.resolve("log"); + } + + @Override + protected void before() throws IOException { + defaultConfig = new UnmodifiableConfiguration(GlobalConfiguration.loadConfiguration(conf.toAbsolutePath().toString())); + final Path originalConfig = conf.resolve(FLINK_CONF_YAML); + final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP); + Files.copy(originalConfig, backupConfig); + filesToDelete.add(new AutoClosablePath(backupConfig)); + } + + @Override + protected void after() { + try { + stopFlinkCluster(); + } catch (IOException e) { + LOG.error("Failure while shutting down Flink cluster.", e); + } + + final Path originalConfig = conf.resolve(FLINK_CONF_YAML); + final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP); + + try { + Files.move(backupConfig, originalConfig, StandardCopyOption.REPLACE_EXISTING); + } catch (IOException e) { + LOG.error("Failed to restore flink-conf.yaml", e); + } + + for (AutoCloseable fileToDelete : filesToDelete) { + try { + fileToDelete.close(); + } catch (Exception e) { + LOG.error("Failure while cleaning up file.", e); + } + } + } + + public void startFlinkCluster() throws IOException { + AutoClosableProcess.runBlocking("Start Flink cluster", bin.resolve("start-cluster.sh").toAbsolutePath().toString()); + + final OkHttpClient client = new OkHttpClient(); + + final Request request = new Request.Builder() + .get() + .url("http://localhost:8081/taskmanagers") + .build(); + + Exception reportedException = null; + for (int retryAttempt = 0; retryAttempt < 30; retryAttempt++) { + try (Response response = client.newCall(request).execute()) { + if (response.isSuccessful()) { + final String json = response.body().string(); + final JsonNode taskManagerList = OBJECT_MAPPER.readTree(json) + .get("taskmanagers"); + + if (taskManagerList != null && taskManagerList.size() > 0) { + LOG.info("Dispatcher REST endpoint is up."); + return; + } + } + } catch (IOException ioe) { + reportedException = ExceptionUtils.firstOrSuppressed(ioe, reportedException); + } + + LOG.info("Waiting for dispatcher REST endpoint to come up..."); + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + } + } + throw new AssertionError("Dispatcher REST endpoint did not start in time.", reportedException); + } + + public void stopFlinkCluster() throws IOException { + AutoClosableProcess.runBlocking("Stop Flink Cluster", bin.resolve("stop-cluster.sh").toAbsolutePath().toString()); + } + + public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException { + final Optional reporterJarOptional = Files.walk(opt) + .filter(path -> path.getFileName().toString().startsWith(jarNamePrefix)) + .findFirst(); + if (reporterJarOptional.isPresent()) { + final Path optReporterJar = reporterJarOptional.get(); + final Path libReporterJar = lib.resolve(optReporterJar.getFileName()); + Files.copy(optReporterJar, libReporterJar); + filesToDelete.add(new AutoClosablePath(libReporterJar)); + } else { + throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePrefix + "."); + } + } + + public void appendConfiguration(Configuration config) throws IOException { + final Configuration mergedConfig = new Configuration(); + mergedConfig.addAll(defaultConfig); + mergedConfig.addAll(config); + + final List configurationLines = mergedConfig.toMap().entrySet().stream() + .map(entry -> entry.getKey() + ": " + entry.getValue()) + .collect(Collectors.toList()); + + Files.write(conf.resolve("flink-conf.yaml"), configurationLines); + } + + public Stream searchAllLogs(Pattern pattern, Function matchProcessor) throws IOException { + final List matches = new ArrayList<>(2); + + try (Stream logFilesStream = Files.list(log)) { + final Iterator logFiles = logFilesStream.iterator(); + while (logFiles.hasNext()) { + final Path logFile = logFiles.next(); + if (!logFile.getFileName().toString().endsWith(".log")) { + // ignore logs for previous runs that have a number suffix + continue; + } + try (BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(logFile.toFile()), StandardCharsets.UTF_8))) { + String line; + while ((line = br.readLine()) != null) { + Matcher matcher = pattern.matcher(line); + if (matcher.matches()) { + matches.add(matchProcessor.apply(matcher)); + } + } + } + } + } + return matches.stream(); + } +} diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml new file mode 100644 index 0000000000000..ddeac14d6de15 --- /dev/null +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml @@ -0,0 +1,104 @@ + + + + + flink-end-to-end-tests + org.apache.flink + 1.7-SNAPSHOT + + + 4.0.0 + + flink-metrics-reporter-prometheus-test + 1.7-SNAPSHOT + + + + org.apache.flink + flink-core + ${project.version} + test + + + org.apache.flink + flink-metrics-prometheus + ${project.version} + test + + + com.squareup.okhttp3 + okhttp + 3.7.0 + test + + + org.apache.flink + flink-shaded-jackson + test + + + org.apache.flink + flink-end-to-end-tests-common + ${project.version} + test + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + + + + + e2e-prometheus + + + e2e-metrics + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + e2e-prometheus + integration-test + + test + + + + **/*ITCase.* + + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java new file mode 100644 index 0000000000000..269754eaf6998 --- /dev/null +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java @@ -0,0 +1,210 @@ +/* + * 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.prometheus.tests; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.prometheus.PrometheusReporter; +import org.apache.flink.tests.util.AutoClosableProcess; +import org.apache.flink.tests.util.CommandLineWrapper; +import org.apache.flink.tests.util.FlinkDistribution; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.OperatingSystem; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking; +import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking; + +/** + * End-to-end test for the PrometheusReporter. + */ +public class PrometheusReporterEndToEndITCase extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static final String PROMETHEUS_VERSION = "2.4.3"; + private static final String PROMETHEUS_FILE_NAME; + + static { + final String base = "prometheus-" + PROMETHEUS_VERSION + '.'; + switch (OperatingSystem.getCurrentOperatingSystem()) { + case MAC_OS: + PROMETHEUS_FILE_NAME = base + "darwin-amd64"; + break; + case WINDOWS: + PROMETHEUS_FILE_NAME = base + "windows-amd64"; + break; + default: + PROMETHEUS_FILE_NAME = base + "linux-amd64"; + break; + } + } + + private static final Pattern LOG_REPORTER_PORT_PATTERN = Pattern.compile(".*Started PrometheusReporter HTTP server on port ([0-9]+).*"); + + @BeforeClass + public static void checkOS() { + Assume.assumeFalse("This test does not run on Windows.", OperatingSystem.isWindows()); + } + + @Rule + public final FlinkDistribution dist = new FlinkDistribution(); + + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + + @Test + public void testReporter() throws Exception { + dist.copyOptJarsToLib("flink-metrics-prometheus"); + + final Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100"); + + dist.appendConfiguration(config); + + final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus"); + final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz"); + final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME); + final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml"); + final Path prometheusBinary = prometheusBinDir.resolve("prometheus"); + Files.createDirectory(tmpPrometheusDir); + + runBlocking( + "Download of Prometheus", + Duration.ofMinutes(5), + CommandLineWrapper + .wget("https://github.com/prometheus/prometheus/releases/download/v" + PROMETHEUS_VERSION + '/' + prometheusArchive.getFileName()) + .targetDir(tmpPrometheusDir) + .build()); + + runBlocking("Extraction of Prometheus archive", + CommandLineWrapper + .tar(prometheusArchive) + .extract() + .zipped() + .targetDir(tmpPrometheusDir) + .build()); + + runBlocking("Set Prometheus scrape interval", + CommandLineWrapper + .sed("s/\\(scrape_interval:\\).*/\\1 1s/", prometheusConfig) + .inPlace() + .build()); + + dist.startFlinkCluster(); + + final List ports = dist + .searchAllLogs(LOG_REPORTER_PORT_PATTERN, matcher -> matcher.group(1)) + .map(Integer::valueOf) + .collect(Collectors.toList()); + + final String scrapeTargets = ports.stream() + .map(port -> "'localhost:" + port + "'") + .collect(Collectors.joining(", ")); + + runBlocking("Set Prometheus scrape targets to (" + scrapeTargets + ")", + CommandLineWrapper + .sed("s/\\(targets:\\).*/\\1 [" + scrapeTargets + "]/", prometheusConfig) + .inPlace() + .build()); + + try (AutoClosableProcess prometheus = runNonBlocking( + "Start Prometheus server", + prometheusBinary.toAbsolutePath().toString(), + "--config.file=" + prometheusConfig.toAbsolutePath(), + "--storage.tsdb.path=" + prometheusBinDir.resolve("data").toAbsolutePath())) { + + final OkHttpClient client = new OkHttpClient(); + + checkMetricAvailability(client, "flink_jobmanager_numRegisteredTaskManagers"); + checkMetricAvailability(client, "flink_taskmanager_Status_Network_TotalMemorySegments"); + } + } + + private static void checkMetricAvailability(final OkHttpClient client, final String metric) throws InterruptedException { + final Request jobManagerRequest = new Request.Builder() + .get() + .url("http://localhost:9090/api/v1/query?query=" + metric) + .build(); + + Exception reportedException = null; + for (int x = 0; x < 30; x++) { + try (Response response = client.newCall(jobManagerRequest).execute()) { + if (response.isSuccessful()) { + final String json = response.body().string(); + + // Sample response: + //{ + // "status": "success", + // "data": { + // "resultType": "vector", + // "result": [{ + // "metric": { + // "__name__": "flink_jobmanager_numRegisteredTaskManagers", + // "host": "localhost", + // "instance": "localhost:9000", + // "job": "prometheus" + // }, + // "value": [1540548500.107, "1"] + // }] + // } + //} + OBJECT_MAPPER.readTree(json) + .get("data") + .get("result") + .get(0) + .get("value") + .get(1).asInt(); + // if we reach this point some value for the given metric was reported to prometheus + return; + } else { + LOG.info("Retrieving metric failed. Retrying... " + response.code() + ":" + response.message()); + Thread.sleep(1000); + } + } catch (Exception e) { + reportedException = ExceptionUtils.firstOrSuppressed(e, reportedException); + Thread.sleep(1000); + } + } + throw new AssertionError("Could not retrieve metric " + metric + " from Prometheus.", reportedException); + } +} diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..f7425cd14c098 --- /dev/null +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/resources/log4j-test.properties @@ -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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +log4j.rootLogger=INFO, testlogger + +# A1 is set to be a ConsoleAppender. +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%m%n diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index e07cf22fa3d87..d5aeca4d64535 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -56,6 +56,8 @@ under the License. flink-streaming-file-sink-test flink-state-evolution-test flink-e2e-test-utils + flink-end-to-end-tests-common + flink-metrics-reporter-prometheus-test @@ -67,6 +69,22 @@ under the License. true + + org.apache.maven.plugins + maven-surefire-plugin + + + + default-test + none + + + integration-tests + none + + + From 380df0559b5bf33792af7fbb486923803d4190a6 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 31 Oct 2018 15:31:18 +0100 Subject: [PATCH 017/359] [FLINK-10720][tests] Add deployment end-to-end stress test with many inflated task deployment desciptors This closes #6994. --- .../DataStreamAllroundTestJobFactory.java | 4 +- .../pom.xml | 81 ++++++++++ .../HeavyDeploymentStressTestProgram.java | 150 ++++++++++++++++++ flink-end-to-end-tests/pom.xml | 1 + flink-end-to-end-tests/run-nightly-tests.sh | 2 + .../test-scripts/test_heavy_deployment.sh | 46 ++++++ 6 files changed, 282 insertions(+), 2 deletions(-) create mode 100644 flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java create mode 100755 flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java index 3c8d0ad537faa..8bd649cc5ebb1 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java @@ -88,7 +88,7 @@ * Total duration is (sliding_window_operator.num_events) * (sequence_generator_source.event_time.clock_progress). * */ -class DataStreamAllroundTestJobFactory { +public class DataStreamAllroundTestJobFactory { private static final ConfigOption TEST_SEMANTICS = ConfigOptions .key("test.semantics") .defaultValue("exactly-once") @@ -201,7 +201,7 @@ class DataStreamAllroundTestJobFactory { .key("tumbling_window_operator.num_events") .defaultValue(20L); - static void setupEnvironment(StreamExecutionEnvironment env, ParameterTool pt) throws Exception { + public static void setupEnvironment(StreamExecutionEnvironment env, ParameterTool pt) throws Exception { // set checkpointing semantics String semantics = pt.get(TEST_SEMANTICS.key(), TEST_SEMANTICS.defaultValue()); diff --git a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml new file mode 100644 index 0000000000000..cc158a412283f --- /dev/null +++ b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml @@ -0,0 +1,81 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.7-SNAPSHOT + .. + + + flink-heavy-deployment-stress-test + flink-heavy-deployment-stress-test + jar + + + + org.apache.flink + flink-streaming-java_2.11 + ${project.version} + + + org.apache.flink + flink-datastream-allround-test + ${project.version} + + + org.apache.flink + flink-test-utils_2.11 + ${project.version} + compile + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + HeavyDeploymentStressTestProgram + package + + shade + + + HeavyDeploymentStressTestProgram + + + org.apache.flink.deployment.HeavyDeploymentStressTestProgram + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java new file mode 100644 index 0000000000000..d65583f98afce --- /dev/null +++ b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java @@ -0,0 +1,150 @@ +/* + * 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.deployment; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.util.Preconditions; + +import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.setupEnvironment; + + +/** + * End-to-end test for heavy deployment descriptors. This test creates a heavy deployment by producing inflated meta + * data for the source's operator state. The state is registered as union state and will be multiplied in deployment. + */ +public class HeavyDeploymentStressTestProgram { + + private static final ConfigOption NUM_LIST_STATES_PER_OP = ConfigOptions + .key("heavy_deployment_test.num_list_states_per_op") + .defaultValue(100); + + private static final ConfigOption NUM_PARTITIONS_PER_LIST_STATE = ConfigOptions + .key("heavy_deployment_test.num_partitions_per_list_state") + .defaultValue(100); + + public static void main(String[] args) throws Exception { + + final ParameterTool pt = ParameterTool.fromArgs(args); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + setupEnvironment(env, pt); + + final int numStates = + pt.getInt(NUM_LIST_STATES_PER_OP.key(), NUM_LIST_STATES_PER_OP.defaultValue()); + final int numPartitionsPerState = + pt.getInt(NUM_PARTITIONS_PER_LIST_STATE.key(), NUM_PARTITIONS_PER_LIST_STATE.defaultValue()); + + Preconditions.checkState(env.getCheckpointInterval() > 0L, "Checkpointing must be enabled for this test!"); + + env.addSource(new SimpleEndlessSourceWithBloatedState(numStates, numPartitionsPerState)).setParallelism(env.getParallelism()) + .addSink(new DiscardingSink<>()).setParallelism(1); + + env.execute("HeavyDeploymentStressTestProgram"); + } + + /** + * Source with dummy operator state that results in inflated meta data. + */ + static class SimpleEndlessSourceWithBloatedState extends RichParallelSourceFunction + implements CheckpointedFunction, CheckpointListener { + + private static final long serialVersionUID = 1L; + + private final int numListStates; + private final int numPartitionsPerListState; + + private transient volatile boolean isRunning; + + /** Flag to induce failure after we have a valid checkpoint. */ + private transient volatile boolean readyToFail; + + SimpleEndlessSourceWithBloatedState(int numListStates, int numPartitionsPerListState) { + this.numListStates = numListStates; + this.numPartitionsPerListState = numPartitionsPerListState; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) { + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + + readyToFail = false; + + if (context.isRestored()) { + isRunning = false; + } else { + isRunning = true; + + OperatorStateStore operatorStateStore = context.getOperatorStateStore(); + for (int i = 0; i < numListStates; ++i) { + + ListStateDescriptor listStateDescriptor = + new ListStateDescriptor<>("test-list-state-" + i, String.class); + + ListState unionListState = + operatorStateStore.getUnionListState(listStateDescriptor); + + for (int j = 0; j < numPartitionsPerListState; ++j) { + unionListState.add(String.valueOf(j)); + } + } + } + } + + @Override + public void run(SourceContext ctx) throws Exception { + while (isRunning) { + + if (readyToFail && getRuntimeContext().getIndexOfThisSubtask() == 0) { + throw new Exception("Artificial failure."); + } + + synchronized (ctx.getCheckpointLock()) { + ctx.collect("test-element"); + } + + Thread.sleep(1); + } + } + + @Override + public void cancel() { + this.isRunning = false; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + readyToFail = true; + } + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index d5aeca4d64535..18befee859cd1 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -58,6 +58,7 @@ under the License. flink-e2e-test-utils flink-end-to-end-tests-common flink-metrics-reporter-prometheus-test + flink-heavy-deployment-stress-test diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 832bdeef93a43..cbd217df2a421 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -128,5 +128,7 @@ run_test "Running Kerberized YARN on Docker test " "$END_TO_END_DIR/test-scripts run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_client.sh" +run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" + printf "\n[PASS] All tests passed\n" exit 0 diff --git a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh new file mode 100755 index 0000000000000..895e4a7f3027e --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh + +CHECKPOINT_DIR="file://$TEST_DATA_DIR/savepoint-e2e-test-chckpt-dir" + +TEST=flink-heavy-deployment-stress-test +TEST_PROGRAM_NAME=HeavyDeploymentStressTestProgram +TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar + +set_conf "taskmanager.heap.mb" "256" # 256Mb x 20TMs = 5Gb total heap + +set_conf "taskmanager.memory.size" "8" # 8Mb +set_conf "taskmanager.network.memory.min" "8mb" +set_conf "taskmanager.network.memory.max" "8mb" +set_conf "taskmanager.memory.segment-size" "8kb" + +set_conf "taskmanager.numberOfTaskSlots" "10" # 10 slots per TM + +start_cluster # this also starts 1TM +start_taskmanagers 19 # 1TM + 19TM = 20TM a 10 slots = 200 slots + +# This call will result in a deployment with state meta data of 200 x 200 x 50 union states x each 75 entries. +# We can scale up the numbers to make the test even heavier. +$FLINK_DIR/bin/flink run ${TEST_PROGRAM_JAR} \ +--environment.max_parallelism 1024 --environment.parallelism 200 \ +--environment.restart_strategy fixed_delay --environment.restart_strategy.fixed_delay.attempts 3 \ +--state_backend.checkpoint_directory ${CHECKPOINT_DIR} \ +--heavy_deployment_test.num_list_states_per_op 50 --heavy_deployment_test.num_partitions_per_list_state 75 From 89a19ec21eb0bd2fc0d3f8ef23c1f0388b512905 Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Tue, 17 Apr 2018 13:45:23 -0700 Subject: [PATCH 018/359] [FLINK-8985] [e2e] Add a Flink CLI end-to-end test This closes #5863. --- flink-end-to-end-tests/flink-api-test/pom.xml | 105 +++++++++ .../runtime/tests/PeriodicStreamingJob.java | 127 ++++++++++ flink-end-to-end-tests/pom.xml | 1 + .../test-scripts/test_cli_api.sh | 216 ++++++++++++++++++ 4 files changed, 449 insertions(+) create mode 100644 flink-end-to-end-tests/flink-api-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java create mode 100755 flink-end-to-end-tests/test-scripts/test_cli_api.sh diff --git a/flink-end-to-end-tests/flink-api-test/pom.xml b/flink-end-to-end-tests/flink-api-test/pom.xml new file mode 100644 index 0000000000000..24a85e02bef7b --- /dev/null +++ b/flink-end-to-end-tests/flink-api-test/pom.xml @@ -0,0 +1,105 @@ + + + + + + flink-end-to-end-tests + org.apache.flink + 1.6-SNAPSHOT + .. + + + 4.0.0 + + flink-api-test + flink-api-test + jar + + + + org.apache.flink + flink-core + ${project.version} + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + org.apache.flink + flink-connector-filesystem_${scala.binary.version} + ${project.version} + + + org.apache.flink + flink-runtime-web_${scala.binary.version} + ${project.version} + test-jar + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + PeriodicStreamingJob + + + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.runtime.tests.PeriodicStreamingJob + + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java b/flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java new file mode 100644 index 0000000000000..ee8a49b858350 --- /dev/null +++ b/flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java @@ -0,0 +1,127 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +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.api.windowing.time.Time; + +import java.util.Collections; +import java.util.List; + +/** + * This is a periodic streaming job that runs for API testing purpose. + * + *

The stream is bounded and will complete after the specified duration. + * + *

Parameters: + * -outputPath Sets the path to where the result data is written. + * -recordsPerSecond Sets the output record frequency. + * -durationInSecond Sets the running duration of the job. + * -offsetInSecond Sets the startup delay before the processing first message. + */ +public class PeriodicStreamingJob { + + public static void main(String[] args) throws Exception { + ParameterTool params = ParameterTool.fromArgs(args); + String outputPath = params.getRequired("outputPath"); + int recordsPerSecond = params.getInt("recordsPerSecond", 10); + int duration = params.getInt("durationInSecond", 60); + int offset = params.getInt("offsetInSecond", 0); + + StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + sEnv.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime); + sEnv.enableCheckpointing(4000); + sEnv.getConfig().setAutoWatermarkInterval(1000); + + // execute a simple pass through program. + PeriodicSourceGenerator generator = new PeriodicSourceGenerator( + recordsPerSecond, duration, offset); + DataStream rows = sEnv.addSource(generator); + + DataStream result = rows + .keyBy(1) + .timeWindow(Time.seconds(5)) + .sum(0); + + result.writeAsText(outputPath + "/result.txt", FileSystem.WriteMode.OVERWRITE) + .setParallelism(1); + + sEnv.execute(); + } + + /** + * Data-generating source function. + */ + public static class PeriodicSourceGenerator implements SourceFunction, ResultTypeQueryable, ListCheckpointed { + private final int sleepMs; + private final int durationMs; + private final int offsetSeconds; + private long ms = 0; + + public PeriodicSourceGenerator(float rowsPerSecond, int durationSeconds, int offsetSeconds) { + this.durationMs = durationSeconds * 1000; + this.sleepMs = (int) (1000 / rowsPerSecond); + this.offsetSeconds = offsetSeconds; + } + + @Override + public void run(SourceContext ctx) throws Exception { + long offsetMs = offsetSeconds * 1000L; + + while (ms < durationMs) { + synchronized (ctx.getCheckpointLock()) { + ctx.collect(new Tuple2<>(ms + offsetMs, "key")); + } + ms += sleepMs; + Thread.sleep(sleepMs); + } + } + + @Override + public void cancel() { } + + @Override + public TypeInformation getProducedType() { + return Types.TUPLE(Types.LONG, Types.STRING); + } + + @Override + public List snapshotState(long checkpointId, long timestamp) { + return Collections.singletonList(ms); + } + + @Override + public void restoreState(List state) { + for (Long l : state) { + ms += l; + } + } + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 18befee859cd1..72cd360b90d32 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -35,6 +35,7 @@ under the License. flink-end-to-end-tests + flink-api-test flink-parent-child-classloading-test flink-dataset-allround-test flink-datastream-allround-test diff --git a/flink-end-to-end-tests/test-scripts/test_cli_api.sh b/flink-end-to-end-tests/test-scripts/test_cli_api.sh new file mode 100755 index 0000000000000..fd9d305ea6369 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_cli_api.sh @@ -0,0 +1,216 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh + +start_cluster +$FLINK_DIR/bin/taskmanager.sh start +$FLINK_DIR/bin/taskmanager.sh start +$FLINK_DIR/bin/taskmanager.sh start + +# Test for CLI commands. +# verify only the return code the content correctness of the API results. +PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar +JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)" +SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\." +JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR="\"pact\": \"(Data Source)\"" +JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR="\"pact\": \"(Data Sink)\"" +JOB_LIST_REGEX_EXTRACTOR_BY_STATUS="([0-9,a-f]*) :" + +EXIT_CODE=0 + +function extract_job_id_from_job_submission_return() { + if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]]; + then + JOB_ID="${BASH_REMATCH[1]}"; + else + JOB_ID="" + fi + echo "$JOB_ID" +} + +function extract_savepoint_path_from_savepoint_return() { + if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]]; + then + SAVEPOINT_PATH="${BASH_REMATCH[1]}"; + else + SAVEPOINT_PATH="" + fi + echo "$SAVEPOINT_PATH" +} + +function extract_valid_pact_from_job_info_return() { + PACT_MATCH=0 + if [[ $1 =~ $JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR ]]; + then + PACT_MATCH=$PACT_MATCH + else + PACT_MATCH=-1 + fi + if [[ $1 =~ $JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR ]]; + then + PACT_MATCH=$PACT_MATCH + else + PACT_MATCH=-1 + fi + echo ${PACT_MATCH} +} + +function extract_valid_job_list_by_type_from_job_list_return() { + JOB_LIST_MATCH=0 + JOB_LIST_REGEX_EXTRACTOR="$JOB_LIST_REGEX_EXTRACTOR_BY_STATUS $2 $3" + if [[ $1 =~ $JOB_LIST_REGEX_EXTRACTOR ]]; + then + JOB_LIST_MATCH=$JOB_LIST_MATCH + else + JOB_LIST_MATCH=-1 + fi + echo ${JOB_LIST_MATCH} +} + +function extract_task_manager_slot_request_count() { + COUNT=`grep "Receive slot request" $FLINK_DIR/log/*taskexecutor*.log | wc -l` + echo $COUNT +} + +function cleanup_cli_test() { + $FLINK_DIR/bin/taskmanager.sh stop-all + + cleanup +} + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Test default job launch with non-detach mode\n" + printf "==============================================================================\n" + eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar" + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Test job launch with complex parameter set\n" + printf "==============================================================================\n" + eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \ + -c org.apache.flink.examples.java.wordcount.WordCount \ + $FLINK_DIR/examples/batch/WordCount.jar \ + --input file:///$FLINK_DIR/README.txt \ + --output file:///${TEST_DATA_DIR}/out/result" + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Validate job launch parallelism configuration\n" + printf "==============================================================================\n" + RECEIVED_TASKMGR_REQUEST=`extract_task_manager_slot_request_count` + # expected 1 from default launch and 4 from complex parameter set. + if [[ $RECEIVED_TASKMGR_REQUEST == 5 ]]; then + EXIT_CODE=0 + else + EXIT_CODE=-1 + fi +fi + +printf "\n==============================================================================\n" +printf "Test information APIs\n" +printf "==============================================================================\n" +if [ $EXIT_CODE == 0 ]; then + RETURN=`$FLINK_DIR/bin/flink info $FLINK_DIR/examples/batch/WordCount.jar` + echo "job info returns: $RETURN" + PACT_MATCH=`extract_valid_pact_from_job_info_return "$RETURN"` + echo "job info regex match: $PACT_MATCH" + if [[ $PACT_MATCH == -1 ]]; then # expect at least a Data Source and a Data Sink pact match + EXIT_CODE=-1 + else + EXIT_CODE=0 + fi +fi + +printf "\n==============================================================================\n" +printf "Test operation on running streaming jobs\n" +printf "==============================================================================\n" +JOB_ID="" +if [ $EXIT_CODE == 0 ]; then + RETURN=`$FLINK_DIR/bin/flink run -d \ + $PERIODIC_JOB_JAR --outputPath file:///${TEST_DATA_DIR}/out/result` + echo "job submission returns: $RETURN" + JOB_ID=`extract_job_id_from_job_submission_return "$RETURN"` + EXIT_CODE=$? # expect matching job id extraction +fi + +printf "\n==============================================================================\n" +printf "Test list API on a streaming job \n" +printf "==============================================================================\n" +if [ $EXIT_CODE == 0 ]; then + RETURN=`$FLINK_DIR/bin/flink list -a` + echo "job list all returns: $RETURN" + JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" ""` + echo "job list all regex match: $JOB_LIST_MATCH" + if [[ $JOB_LIST_MATCH == -1 ]]; then # expect match for all job + EXIT_CODE=-1 + else + EXIT_CODE=0 + fi +fi +if [ $EXIT_CODE == 0 ]; then + RETURN=`$FLINK_DIR/bin/flink list -r` + echo "job list running returns: $RETURN" + JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" "\(RUNNING\)"` + echo "job list running regex match: $JOB_LIST_MATCH" + if [[ $JOB_LIST_MATCH == -1 ]]; then # expect match for running job + EXIT_CODE=-1 + else + EXIT_CODE=0 + fi +fi +if [ $EXIT_CODE == 0 ]; then + RETURN=`$FLINK_DIR/bin/flink list -s` + echo "job list scheduled returns: $RETURN" + JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" "\(CREATED\)"` + echo "job list scheduled regex match: $JOB_LIST_MATCH" + if [[ $JOB_LIST_MATCH == -1 ]]; then # expect no match for scheduled job + EXIT_CODE=0 + else + EXIT_CODE=-1 + fi +fi + +printf "\n==============================================================================\n" +printf "Test canceling a running streaming jobs\n" +printf "==============================================================================\n" +if [ $EXIT_CODE == 0 ]; then + eval "$FLINK_DIR/bin/flink cancel ${JOB_ID}" + EXIT_CODE=$? +fi + +printf "\n==============================================================================\n" +printf "Cleaning up... \n" +printf "==============================================================================\n" +trap cleanup_cli_test INT +trap cleanup_cli_test EXIT + +if [ $EXIT_CODE == 0 ]; + then + echo "All CLI test passed!"; + else + echo "CLI test failed: $EXIT_CODE"; + PASS="" + exit 1 +fi From a73f979ea56d03a80d620a76bfd9078818d8c6d5 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 6 Nov 2018 17:15:20 +0100 Subject: [PATCH 019/359] [FLINK-8985] [e2e] Add additional checks and error messages to CLI test --- .../pom.xml | 44 +----- .../tests/PeriodicStreamingJob.java | 14 +- flink-end-to-end-tests/pom.xml | 2 +- flink-end-to-end-tests/run-nightly-tests.sh | 2 + .../{test_cli_api.sh => test_cli.sh} | 127 ++++++++---------- 5 files changed, 72 insertions(+), 117 deletions(-) rename flink-end-to-end-tests/{flink-api-test => flink-cli-test}/pom.xml (62%) rename flink-end-to-end-tests/{flink-api-test/src/main/java/org/apache/flink/runtime => flink-cli-test/src/main/java/org/apache/flink/streaming}/tests/PeriodicStreamingJob.java (92%) rename flink-end-to-end-tests/test-scripts/{test_cli_api.sh => test_cli.sh} (68%) diff --git a/flink-end-to-end-tests/flink-api-test/pom.xml b/flink-end-to-end-tests/flink-cli-test/pom.xml similarity index 62% rename from flink-end-to-end-tests/flink-api-test/pom.xml rename to flink-end-to-end-tests/flink-cli-test/pom.xml index 24a85e02bef7b..ebb42322e87aa 100644 --- a/flink-end-to-end-tests/flink-api-test/pom.xml +++ b/flink-end-to-end-tests/flink-cli-test/pom.xml @@ -24,37 +24,22 @@ flink-end-to-end-tests org.apache.flink - 1.6-SNAPSHOT + 1.7-SNAPSHOT .. 4.0.0 - flink-api-test - flink-api-test + flink-cli-test_${scala.binary.version} + flink-cli-test jar - - org.apache.flink - flink-core - ${project.version} - org.apache.flink flink-streaming-java_${scala.binary.version} ${project.version} - - - org.apache.flink - flink-connector-filesystem_${scala.binary.version} - ${project.version} - - - org.apache.flink - flink-runtime-web_${scala.binary.version} - ${project.version} - test-jar + provided @@ -63,35 +48,18 @@ org.apache.maven.plugins maven-shade-plugin - 3.0.0 + PeriodicStreamingJob package shade PeriodicStreamingJob - - - com.google.code.findbugs:jsr305 - org.slf4j:* - log4j:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - org.apache.flink.runtime.tests.PeriodicStreamingJob + org.apache.flink.streaming.tests.PeriodicStreamingJob diff --git a/flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java b/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java similarity index 92% rename from flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java rename to flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java index ee8a49b858350..7cabf849aaf03 100644 --- a/flink-end-to-end-tests/flink-api-test/src/main/java/org/apache/flink/runtime/tests/PeriodicStreamingJob.java +++ b/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.tests; +package org.apache.flink.streaming.tests; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; @@ -36,7 +36,7 @@ import java.util.List; /** - * This is a periodic streaming job that runs for API testing purpose. + * This is a periodic streaming job that runs for CLI testing purposes. * *

The stream is bounded and will complete after the specified duration. * @@ -44,7 +44,7 @@ * -outputPath Sets the path to where the result data is written. * -recordsPerSecond Sets the output record frequency. * -durationInSecond Sets the running duration of the job. - * -offsetInSecond Sets the startup delay before the processing first message. + * -offsetInSecond Sets the startup delay before processing the first message. */ public class PeriodicStreamingJob { @@ -63,9 +63,9 @@ public static void main(String[] args) throws Exception { // execute a simple pass through program. PeriodicSourceGenerator generator = new PeriodicSourceGenerator( recordsPerSecond, duration, offset); - DataStream rows = sEnv.addSource(generator); + DataStream rows = sEnv.addSource(generator); - DataStream result = rows + DataStream result = rows .keyBy(1) .timeWindow(Time.seconds(5)) .sum(0); @@ -105,7 +105,9 @@ public void run(SourceContext ctx) throws Exception { } @Override - public void cancel() { } + public void cancel() { + // nothing to do + } @Override public TypeInformation getProducedType() { diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 72cd360b90d32..c6644e25a9460 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -35,7 +35,7 @@ under the License. flink-end-to-end-tests - flink-api-test + flink-cli-test flink-parent-child-classloading-test flink-dataset-allround-test flink-datastream-allround-test diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index cbd217df2a421..5a6363c324489 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -49,6 +49,8 @@ echo "Flink distribution directory: $FLINK_DIR" # those checks are disabled, one should take care that a proper checks are performed in the tests itself that ensure that the test finished # in an expected state. +run_test "Flink CLI end-to-end test" "$END_TO_END_DIR/test-scripts/test_cli.sh" + run_test "ConnectedComponents iterations with high parallelism end-to-end test" "$END_TO_END_DIR/test-scripts/test_high_parallelism_iterations.sh 25" run_test "Queryable state (rocksdb) end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state.sh rocksdb" diff --git a/flink-end-to-end-tests/test-scripts/test_cli_api.sh b/flink-end-to-end-tests/test-scripts/test_cli.sh similarity index 68% rename from flink-end-to-end-tests/test-scripts/test_cli_api.sh rename to flink-end-to-end-tests/test-scripts/test_cli.sh index fd9d305ea6369..9e42b3b645758 100755 --- a/flink-end-to-end-tests/test-scripts/test_cli_api.sh +++ b/flink-end-to-end-tests/test-scripts/test_cli.sh @@ -19,16 +19,15 @@ source "$(dirname "$0")"/common.sh +TEST_PROGRAM_JAR=$END_TO_END_DIR/flink-cli-test/target/PeriodicStreamingJob.jar + start_cluster $FLINK_DIR/bin/taskmanager.sh start $FLINK_DIR/bin/taskmanager.sh start $FLINK_DIR/bin/taskmanager.sh start -# Test for CLI commands. -# verify only the return code the content correctness of the API results. -PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar +# CLI regular expressions JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)" -SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\." JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR="\"pact\": \"(Data Source)\"" JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR="\"pact\": \"(Data Sink)\"" JOB_LIST_REGEX_EXTRACTOR_BY_STATUS="([0-9,a-f]*) :" @@ -45,16 +44,6 @@ function extract_job_id_from_job_submission_return() { echo "$JOB_ID" } -function extract_savepoint_path_from_savepoint_return() { - if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]]; - then - SAVEPOINT_PATH="${BASH_REMATCH[1]}"; - else - SAVEPOINT_PATH="" - fi - echo "$SAVEPOINT_PATH" -} - function extract_valid_pact_from_job_info_return() { PACT_MATCH=0 if [[ $1 =~ $JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR ]]; @@ -89,57 +78,59 @@ function extract_task_manager_slot_request_count() { echo $COUNT } -function cleanup_cli_test() { - $FLINK_DIR/bin/taskmanager.sh stop-all - - cleanup -} +printf "\n==============================================================================\n" +printf "Test default job launch with non-detach mode\n" +printf "==============================================================================\n" +RESULT=`$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar` +EXIT_CODE=$? +echo "$RESULT" -if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Test default job launch with non-detach mode\n" - printf "==============================================================================\n" - eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar" - EXIT_CODE=$? +if [[ $RESULT != *"(java.util.ArrayList) [170 elements]"* ]];then + echo "[FAIL] Invalid accumulator result." + EXIT_CODE=1 fi if [ $EXIT_CODE == 0 ]; then printf "\n==============================================================================\n" printf "Test job launch with complex parameter set\n" printf "==============================================================================\n" - eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \ + eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q \ -c org.apache.flink.examples.java.wordcount.WordCount \ $FLINK_DIR/examples/batch/WordCount.jar \ --input file:///$FLINK_DIR/README.txt \ - --output file:///${TEST_DATA_DIR}/out/result" + --output file:///${TEST_DATA_DIR}/result1" EXIT_CODE=$? fi if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Validate job launch parallelism configuration\n" - printf "==============================================================================\n" + ROW_COUNT=`cat ${TEST_DATA_DIR}/result1/* | wc -l` + if [ $((ROW_COUNT)) -ne 111 ]; then + echo "[FAIL] Unexpected number of rows in output." + echo "Found: $ROW_COUNT" + EXIT_CODE=1 + fi +fi + +if [ $EXIT_CODE == 0 ]; then RECEIVED_TASKMGR_REQUEST=`extract_task_manager_slot_request_count` # expected 1 from default launch and 4 from complex parameter set. - if [[ $RECEIVED_TASKMGR_REQUEST == 5 ]]; then - EXIT_CODE=0 - else - EXIT_CODE=-1 + if [[ $RECEIVED_TASKMGR_REQUEST != 5 ]]; then + echo "[FAIL] Unexpected task manager slot count." + echo "Received slots: $RECEIVED_TASKMGR_REQUEST" + EXIT_CODE=1 fi fi printf "\n==============================================================================\n" -printf "Test information APIs\n" +printf "Test CLI information\n" printf "==============================================================================\n" if [ $EXIT_CODE == 0 ]; then RETURN=`$FLINK_DIR/bin/flink info $FLINK_DIR/examples/batch/WordCount.jar` - echo "job info returns: $RETURN" + echo "$RETURN" PACT_MATCH=`extract_valid_pact_from_job_info_return "$RETURN"` - echo "job info regex match: $PACT_MATCH" if [[ $PACT_MATCH == -1 ]]; then # expect at least a Data Source and a Data Sink pact match - EXIT_CODE=-1 - else - EXIT_CODE=0 + echo "[FAIL] Data source and/or sink are missing." + EXIT_CODE=1 fi fi @@ -148,9 +139,8 @@ printf "Test operation on running streaming jobs\n" printf "==============================================================================\n" JOB_ID="" if [ $EXIT_CODE == 0 ]; then - RETURN=`$FLINK_DIR/bin/flink run -d \ - $PERIODIC_JOB_JAR --outputPath file:///${TEST_DATA_DIR}/out/result` - echo "job submission returns: $RETURN" + RETURN=`$FLINK_DIR/bin/flink run -d $TEST_PROGRAM_JAR --outputPath file:///${TEST_DATA_DIR}/result2` + echo "$RETURN" JOB_ID=`extract_job_id_from_job_submission_return "$RETURN"` EXIT_CODE=$? # expect matching job id extraction fi @@ -160,35 +150,31 @@ printf "Test list API on a streaming job \n" printf "==============================================================================\n" if [ $EXIT_CODE == 0 ]; then RETURN=`$FLINK_DIR/bin/flink list -a` - echo "job list all returns: $RETURN" + echo "$RETURN" JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" ""` - echo "job list all regex match: $JOB_LIST_MATCH" if [[ $JOB_LIST_MATCH == -1 ]]; then # expect match for all job - EXIT_CODE=-1 - else - EXIT_CODE=0 + echo "[FAIL] Unexpected 'Flink Streaming Job' list." + EXIT_CODE=1 fi fi + if [ $EXIT_CODE == 0 ]; then RETURN=`$FLINK_DIR/bin/flink list -r` - echo "job list running returns: $RETURN" + echo "$RETURN" JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" "\(RUNNING\)"` - echo "job list running regex match: $JOB_LIST_MATCH" if [[ $JOB_LIST_MATCH == -1 ]]; then # expect match for running job - EXIT_CODE=-1 - else - EXIT_CODE=0 + echo "[FAIL] Unexpected 'Flink Streaming Job' 'RUNNING' list." + EXIT_CODE=1 fi fi + if [ $EXIT_CODE == 0 ]; then RETURN=`$FLINK_DIR/bin/flink list -s` - echo "job list scheduled returns: $RETURN" + echo "$RETURN" JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" "\(CREATED\)"` - echo "job list scheduled regex match: $JOB_LIST_MATCH" - if [[ $JOB_LIST_MATCH == -1 ]]; then # expect no match for scheduled job - EXIT_CODE=0 - else - EXIT_CODE=-1 + if [[ $JOB_LIST_MATCH != -1 ]]; then # expect no match for scheduled job + echo "[FAIL] Unexpected 'Flink Streaming Job' 'CREATED' list." + EXIT_CODE=1 fi fi @@ -200,17 +186,14 @@ if [ $EXIT_CODE == 0 ]; then EXIT_CODE=$? fi -printf "\n==============================================================================\n" -printf "Cleaning up... \n" -printf "==============================================================================\n" -trap cleanup_cli_test INT -trap cleanup_cli_test EXIT - -if [ $EXIT_CODE == 0 ]; - then - echo "All CLI test passed!"; - else - echo "CLI test failed: $EXIT_CODE"; - PASS="" - exit 1 +if [ $EXIT_CODE == 0 ]; then + RETURN=`$FLINK_DIR/bin/flink list -a` + echo "$RETURN" + JOB_LIST_MATCH=`extract_valid_job_list_by_type_from_job_list_return "$RETURN" "Flink Streaming Job" "\(CANCELED\)"` + if [[ $JOB_LIST_MATCH == -1 ]]; then # expect match for canceled job + echo "[FAIL] Unexpected 'Flink Streaming Job' 'CANCELED' list." + EXIT_CODE=1 + fi fi + +exit $EXIT_CODE From 573eebbe437ee90cb0841354ee96dd0f4594cc8d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 7 Nov 2018 10:05:44 +0100 Subject: [PATCH 020/359] [hotfix] [e2e] Add missing Kubernetes test --- flink-end-to-end-tests/run-nightly-tests.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 5a6363c324489..afeb139f5156c 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -128,6 +128,8 @@ run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scrip run_test "Running Kerberized YARN on Docker test " "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" +run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" + run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_client.sh" run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" From d4f3d4a792cc48439a9e92ec4e0ae4e6ed09da8b Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 7 Nov 2018 10:19:51 +0100 Subject: [PATCH 021/359] [hotfix] [e2e] Categorize nightly tests --- flink-end-to-end-tests/run-nightly-tests.sh | 82 +++++++++++++-------- 1 file changed, 53 insertions(+), 29 deletions(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index afeb139f5156c..7e15870f1d31f 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -49,24 +49,9 @@ echo "Flink distribution directory: $FLINK_DIR" # those checks are disabled, one should take care that a proper checks are performed in the tests itself that ensure that the test finished # in an expected state. -run_test "Flink CLI end-to-end test" "$END_TO_END_DIR/test-scripts/test_cli.sh" - -run_test "ConnectedComponents iterations with high parallelism end-to-end test" "$END_TO_END_DIR/test-scripts/test_high_parallelism_iterations.sh 25" - -run_test "Queryable state (rocksdb) end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state.sh rocksdb" -run_test "Queryable state (rocksdb) with TM restart end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state_restart_tm.sh" "skip_check_exceptions" - -run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions" - -run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false" "skip_check_exceptions" -run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false" "skip_check_exceptions" -run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false" "skip_check_exceptions" -run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true" "skip_check_exceptions" - -run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false" "skip_check_exceptions" -run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false" "skip_check_exceptions" -run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false" "skip_check_exceptions" -run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true" "skip_check_exceptions" +################################################################################ +# Checkpointing tests +################################################################################ run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true" run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false" @@ -99,6 +84,43 @@ run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) e run_test "Resuming Externalized Checkpoint after terminal failure (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 rocks true false true" run_test "Resuming Externalized Checkpoint after terminal failure (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 rocks true true true" +################################################################################ +# Docker +################################################################################ + +run_test "Running Kerberized YARN on Docker test " "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" + +################################################################################ +# High Availability +################################################################################ + +run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions" + +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false" "skip_check_exceptions" +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false" "skip_check_exceptions" +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false" "skip_check_exceptions" +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true" "skip_check_exceptions" + +run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false" "skip_check_exceptions" +run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true" "skip_check_exceptions" + +################################################################################ +# Kubernetes +################################################################################ + +run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" + +################################################################################ +# Miscellaneous +################################################################################ + +run_test "Flink CLI end-to-end test" "$END_TO_END_DIR/test-scripts/test_cli.sh" + +run_test "Queryable state (rocksdb) end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state.sh rocksdb" +run_test "Queryable state (rocksdb) with TM restart end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state_restart_tm.sh" "skip_check_exceptions" + run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh" run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh" "skip_check_exceptions" run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh" "skip_check_exceptions" @@ -106,13 +128,6 @@ run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/tes run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" -run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false false" "skip_check_exceptions" -run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false true" "skip_check_exceptions" -run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false false" "skip_check_exceptions" -run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true false" "skip_check_exceptions" -run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false true" "skip_check_exceptions" -run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true true" "skip_check_exceptions" - run_test "Elasticsearch (v1.7.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz" run_test "Elasticsearch (v2.3.5) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz" run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" @@ -126,13 +141,22 @@ run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_D run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file" run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh rocks" -run_test "Running Kerberized YARN on Docker test " "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" - -run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" - run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_client.sh" run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" +run_test "ConnectedComponents iterations with high parallelism end-to-end test" "$END_TO_END_DIR/test-scripts/test_high_parallelism_iterations.sh 25" + +################################################################################ +# Sticky Scheduling +################################################################################ + +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false false" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false true" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false false" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true false" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false true" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true true" "skip_check_exceptions" + printf "\n[PASS] All tests passed\n" exit 0 From 090119ac96f7c62e592f020edf583b032f823add Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 2 Nov 2018 11:48:46 +0100 Subject: [PATCH 022/359] [FLINK-10353][kafka] Support change of transactional semantics in Kafka Producer with existing state This closes #7010. --- .../kafka/FlinkKafkaProducer011.java | 92 ++++++++----------- .../kafka/internal/FlinkKafkaProducer.java | 1 + .../kafka/FlinkKafkaProducer011ITCase.java | 68 ++++++++++++-- .../kafka/KafkaProducerTestBase.java | 2 +- .../connectors/kafka/FlinkKafkaProducer.java | 92 ++++++++----------- .../kafka/FlinkKafkaProducerITCase.java | 62 ++++++++++++- .../sink/TwoPhaseCommitSinkFunction.java | 6 +- 7 files changed, 195 insertions(+), 128 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 1cf4acf54064e..c7f84c36b6ae8 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -689,7 +689,7 @@ protected KafkaTransactionState beginTransaction() throws FlinkKafka011Exception if (currentTransaction != null && currentTransaction.producer != null) { return new KafkaTransactionState(currentTransaction.producer); } - return new KafkaTransactionState(initProducer(true)); + return new KafkaTransactionState(initNonTransactionalProducer(true)); default: throw new UnsupportedOperationException("Not implemented semantic"); } @@ -712,73 +712,46 @@ protected void preCommit(KafkaTransactionState transaction) throws FlinkKafka011 @Override protected void commit(KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - transaction.producer.commitTransaction(); - recycleTransactionalProducer(transaction.producer); - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + if (transaction.isTransactional()) { + transaction.producer.commitTransaction(); + recycleTransactionalProducer(transaction.producer); } } @Override protected void recoverAndCommit(KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - try (FlinkKafkaProducer producer = - initTransactionalProducer(transaction.transactionalId, false)) { - producer.resumeTransaction(transaction.producerId, transaction.epoch); - producer.commitTransaction(); - } - catch (InvalidTxnStateException | ProducerFencedException ex) { - // That means we have committed this transaction before. - LOG.warn("Encountered error {} while recovering transaction {}. " + + if (transaction.isTransactional()) { + try ( + FlinkKafkaProducer producer = + initTransactionalProducer(transaction.transactionalId, false)) { + producer.resumeTransaction(transaction.producerId, transaction.epoch); + producer.commitTransaction(); + } catch (InvalidTxnStateException | ProducerFencedException ex) { + // That means we have committed this transaction before. + LOG.warn("Encountered error {} while recovering transaction {}. " + "Presumably this transaction has been already committed before", - ex, - transaction); - } - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + ex, + transaction); + } } } @Override protected void abort(KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - transaction.producer.abortTransaction(); - recycleTransactionalProducer(transaction.producer); - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + if (transaction.isTransactional()) { + transaction.producer.abortTransaction(); + recycleTransactionalProducer(transaction.producer); } } @Override protected void recoverAndAbort(KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - try (FlinkKafkaProducer producer = - initTransactionalProducer(transaction.transactionalId, false)) { - producer.initTransactions(); - } - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + if (transaction.isTransactional()) { + try ( + FlinkKafkaProducer producer = + initTransactionalProducer(transaction.transactionalId, false)) { + producer.initTransactions(); + } } } @@ -905,9 +878,7 @@ private void cleanUpUserContext() { private void resetAvailableTransactionalIdsPool(Collection transactionalIds) { availableTransactionalIds.clear(); - for (String transactionalId : transactionalIds) { - availableTransactionalIds.add(transactionalId); - } + availableTransactionalIds.addAll(transactionalIds); } // ----------------------------------- Utilities -------------------------- @@ -957,6 +928,11 @@ private FlinkKafkaProducer initTransactionalProducer(String tran return initProducer(registerMetrics); } + private FlinkKafkaProducer initNonTransactionalProducer(boolean registerMetrics) { + producerConfig.remove("transactional.id"); + return initProducer(registerMetrics); + } + private FlinkKafkaProducer initProducer(boolean registerMetrics) { FlinkKafkaProducer producer = new FlinkKafkaProducer<>(this.producerConfig); @@ -1075,7 +1051,7 @@ static class KafkaTransactionState { } KafkaTransactionState( - String transactionalId, + @Nullable String transactionalId, long producerId, short epoch, FlinkKafkaProducer producer) { @@ -1085,6 +1061,10 @@ static class KafkaTransactionState { this.producer = producer; } + boolean isTransactional() { + return transactionalId != null; + } + @Override public String toString() { return String.format( diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java index 8faff38749fbd..fa672f03ccd8b 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java @@ -210,6 +210,7 @@ public void resumeTransaction(long producerId, short epoch) { } } + @Nullable public String getTransactionalId() { return transactionalId; } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index 57b7e77dc7f8f..39a6273318ed7 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -48,11 +48,13 @@ import java.util.stream.IntStream; import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.Semantic; +import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.Semantic.AT_LEAST_ONCE; +import static org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.Semantic.EXACTLY_ONCE; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; /** * IT cases for the {@link FlinkKafkaProducer011}. @@ -87,7 +89,7 @@ public void resourceCleanUpNone() throws Exception { @Test public void resourceCleanUpAtLeastOnce() throws Exception { - resourceCleanUp(Semantic.AT_LEAST_ONCE); + resourceCleanUp(AT_LEAST_ONCE); } /** @@ -183,7 +185,7 @@ public void testFlinkKafkaProducer011FailBeforeNotify() throws Exception { try { testHarness.processElement(44, 4); testHarness.snapshot(2, 5); - assertFalse(true); + fail(); } catch (Exception ex) { // expected @@ -216,7 +218,7 @@ public void testFlinkKafkaProducer011FailTransactionCoordinatorBeforeNotify() th topic, integerKeyedSerializationSchema, properties, - Semantic.EXACTLY_ONCE); + EXACTLY_ONCE); OneInputStreamOperatorTestHarness testHarness1 = new OneInputStreamOperatorTestHarness<>( new StreamSink<>(kafkaProducer), @@ -367,7 +369,7 @@ public void testScaleDownBeforeFirstCheckpoint() throws Exception { preScaleDownParallelism, preScaleDownParallelism, subtaskIndex, - Semantic.EXACTLY_ONCE); + EXACTLY_ONCE); preScaleDownOperator.setup(); preScaleDownOperator.open(); @@ -382,7 +384,7 @@ public void testScaleDownBeforeFirstCheckpoint() throws Exception { // there might not be any close) // After previous failure simulate restarting application with smaller parallelism - OneInputStreamOperatorTestHarness postScaleDownOperator1 = createTestHarness(topic, 1, 1, 0, Semantic.EXACTLY_ONCE); + OneInputStreamOperatorTestHarness postScaleDownOperator1 = createTestHarness(topic, 1, 1, 0, EXACTLY_ONCE); postScaleDownOperator1.setup(); postScaleDownOperator1.open(); @@ -483,7 +485,7 @@ private List repartitionAndExecute( for (int subtaskIndex = 0; subtaskIndex < parallelism; subtaskIndex++) { OneInputStreamOperatorTestHarness testHarness = - createTestHarness(topic, maxParallelism, parallelism, subtaskIndex, Semantic.EXACTLY_ONCE); + createTestHarness(topic, maxParallelism, parallelism, subtaskIndex, EXACTLY_ONCE); testHarnesses.add(testHarness); testHarness.setup(); @@ -566,6 +568,50 @@ public void testRunOutOfProducersInThePool() throws Exception { deleteTestTopic(topic); } + @Test + public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { + String topic = "testMigrateFromAtLeastOnceToExactlyOnce"; + testRecoverWithChangeSemantics(topic, AT_LEAST_ONCE, EXACTLY_ONCE); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 44, 45), 30_000L); + deleteTestTopic(topic); + } + + @Test + public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { + String topic = "testMigrateFromExactlyOnceToAtLeastOnce"; + testRecoverWithChangeSemantics(topic, EXACTLY_ONCE, AT_LEAST_ONCE); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 45, 46, 47), 30_000L); + deleteTestTopic(topic); + } + + private void testRecoverWithChangeSemantics( + String topic, + Semantic fromSemantic, + Semantic toSemantic) throws Exception { + OperatorSubtaskState producerSnapshot; + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic, fromSemantic)) { + testHarness.setup(); + testHarness.open(); + testHarness.processElement(42, 0); + testHarness.snapshot(0, 1); + testHarness.processElement(43, 2); + testHarness.notifyOfCompletedCheckpoint(0); + producerSnapshot = testHarness.snapshot(1, 3); + testHarness.processElement(44, 4); + } + + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic, toSemantic)) { + testHarness.setup(); + testHarness.initializeState(producerSnapshot); + testHarness.open(); + testHarness.processElement(45, 7); + testHarness.snapshot(2, 8); + testHarness.processElement(46, 9); + testHarness.notifyOfCompletedCheckpoint(2); + testHarness.processElement(47, 9); + } + } + // shut down a Kafka broker private void failBroker(int brokerId) { KafkaServer toShutDown = null; @@ -604,7 +650,13 @@ private void closeIgnoringProducerFenced(AutoCloseable autoCloseable) throws Exc } private OneInputStreamOperatorTestHarness createTestHarness(String topic) throws Exception { - return createTestHarness(topic, 1, 1, 0, Semantic.EXACTLY_ONCE); + return createTestHarness(topic, Semantic.EXACTLY_ONCE); + } + + private OneInputStreamOperatorTestHarness createTestHarness( + String topic, + Semantic semantic) throws Exception { + return createTestHarness(topic, 1, 1, 0, semantic); } private OneInputStreamOperatorTestHarness createTestHarness( diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 69eb94a2e8f72..cf68c3de77635 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -329,7 +329,7 @@ protected void testExactlyOnce(boolean regularSink, int sinksCount) throws Excep } TypeInformationSerializationSchema schema = new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig()); - KeyedSerializationSchema keyedSerializationSchema = new KeyedSerializationSchemaWrapper(schema); + KeyedSerializationSchema keyedSerializationSchema = new KeyedSerializationSchemaWrapper<>(schema); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500); diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index 0ac2f906bde53..df1a4b5727f91 100644 --- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -691,7 +691,7 @@ protected FlinkKafkaProducer.KafkaTransactionState beginTransaction() throws Fli if (currentTransaction != null && currentTransaction.producer != null) { return new FlinkKafkaProducer.KafkaTransactionState(currentTransaction.producer); } - return new FlinkKafkaProducer.KafkaTransactionState(initProducer(true)); + return new FlinkKafkaProducer.KafkaTransactionState(initNonTransactionalProducer(true)); default: throw new UnsupportedOperationException("Not implemented semantic"); } @@ -714,73 +714,46 @@ protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) t @Override protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - transaction.producer.commitTransaction(); - recycleTransactionalProducer(transaction.producer); - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + if (transaction.isTransactional()) { + transaction.producer.commitTransaction(); + recycleTransactionalProducer(transaction.producer); } } @Override protected void recoverAndCommit(FlinkKafkaProducer.KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - try (FlinkKafkaInternalProducer producer = + if (transaction.isTransactional()) { + try ( + FlinkKafkaInternalProducer producer = initTransactionalProducer(transaction.transactionalId, false)) { - producer.resumeTransaction(transaction.producerId, transaction.epoch); - producer.commitTransaction(); - } - catch (InvalidTxnStateException | ProducerFencedException ex) { - // That means we have committed this transaction before. - LOG.warn("Encountered error {} while recovering transaction {}. " + - "Presumably this transaction has been already committed before", - ex, - transaction); - } - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + producer.resumeTransaction(transaction.producerId, transaction.epoch); + producer.commitTransaction(); + } catch (InvalidTxnStateException | ProducerFencedException ex) { + // That means we have committed this transaction before. + LOG.warn("Encountered error {} while recovering transaction {}. " + + "Presumably this transaction has been already committed before", + ex, + transaction); + } } } @Override protected void abort(FlinkKafkaProducer.KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - transaction.producer.abortTransaction(); - recycleTransactionalProducer(transaction.producer); - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + if (transaction.isTransactional()) { + transaction.producer.abortTransaction(); + recycleTransactionalProducer(transaction.producer); } } @Override protected void recoverAndAbort(FlinkKafkaProducer.KafkaTransactionState transaction) { - switch (semantic) { - case EXACTLY_ONCE: - try (FlinkKafkaInternalProducer producer = + if (transaction.isTransactional()) { + try ( + FlinkKafkaInternalProducer producer = initTransactionalProducer(transaction.transactionalId, false)) { - producer.initTransactions(); - } - break; - case AT_LEAST_ONCE: - case NONE: - break; - default: - throw new UnsupportedOperationException("Not implemented semantic"); + producer.initTransactions(); + } } } @@ -895,7 +868,7 @@ protected void finishRecoveringContext() { LOG.info("Recovered transactionalIds {}", getUserContext().get().transactionalIds); } - protected FlinkKafkaInternalProducer createProducer() { + protected FlinkKafkaInternalProducer createProducer() { return new FlinkKafkaInternalProducer<>(this.producerConfig); } @@ -911,9 +884,7 @@ private void cleanUpUserContext() { private void resetAvailableTransactionalIdsPool(Collection transactionalIds) { availableTransactionalIds.clear(); - for (String transactionalId : transactionalIds) { - availableTransactionalIds.add(transactionalId); - } + availableTransactionalIds.addAll(transactionalIds); } // ----------------------------------- Utilities -------------------------- @@ -963,6 +934,11 @@ private FlinkKafkaInternalProducer initTransactionalProducer(Str return initProducer(registerMetrics); } + private FlinkKafkaInternalProducer initNonTransactionalProducer(boolean registerMetrics) { + producerConfig.remove("transactional.id"); + return initProducer(registerMetrics); + } + private FlinkKafkaInternalProducer initProducer(boolean registerMetrics) { FlinkKafkaInternalProducer producer = createProducer(); @@ -1081,7 +1057,7 @@ static class KafkaTransactionState { } KafkaTransactionState( - String transactionalId, + @Nullable String transactionalId, long producerId, short epoch, FlinkKafkaInternalProducer producer) { @@ -1091,6 +1067,10 @@ static class KafkaTransactionState { this.producer = producer; } + boolean isTransactional() { + return transactionalId != null; + } + @Override public String toString() { return String.format( diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index b14b3e2524bfc..6070930ae17d9 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -49,8 +49,8 @@ import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; /** * IT cases for the {@link FlinkKafkaProducer}. @@ -163,7 +163,7 @@ public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exceptio } @Test - public void testFlinkKafkaProducer10FailBeforeNotify() throws Exception { + public void testFlinkKafkaProducerFailBeforeNotify() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic); @@ -181,7 +181,7 @@ public void testFlinkKafkaProducer10FailBeforeNotify() throws Exception { try { testHarness.processElement(44, 4); testHarness.snapshot(2, 5); - assertFalse(true); + fail(); } catch (Exception ex) { // expected @@ -205,7 +205,7 @@ public void testFlinkKafkaProducer10FailBeforeNotify() throws Exception { } @Test - public void testFlinkKafkaProducer10FailTransactionCoordinatorBeforeNotify() throws Exception { + public void testFlinkKafkaProducerFailTransactionCoordinatorBeforeNotify() throws Exception { String topic = "flink-kafka-producer-fail-transaction-coordinator-before-notify"; Properties properties = createProperties(); @@ -564,6 +564,52 @@ public void testRunOutOfProducersInThePool() throws Exception { deleteTestTopic(topic); } + @Test + public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { + String topic = "testMigrateFromAtLeastOnceToExactlyOnce"; + testRecoverWithChangeSemantics(topic, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE, FlinkKafkaProducer.Semantic.EXACTLY_ONCE); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 44, 45), 30_000L); + deleteTestTopic(topic); + } + + @Test + public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { + String topic = "testMigrateFromExactlyOnceToAtLeastOnce"; + testRecoverWithChangeSemantics(topic, FlinkKafkaProducer.Semantic.EXACTLY_ONCE, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 45, 46, 47), 30_000L); + deleteTestTopic(topic); + } + + private void testRecoverWithChangeSemantics( + String topic, + FlinkKafkaProducer.Semantic fromSemantic, + FlinkKafkaProducer.Semantic toSemantic) throws Exception { + OperatorSubtaskState producerSnapshot; + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic, fromSemantic)) { + testHarness.setup(); + testHarness.open(); + testHarness.processElement(42, 0); + testHarness.snapshot(0, 1); + testHarness.processElement(43, 2); + testHarness.notifyOfCompletedCheckpoint(0); + producerSnapshot = testHarness.snapshot(1, 3); + testHarness.processElement(44, 4); + } + + try (OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic, toSemantic)) { + testHarness.setup(); + testHarness.initializeState(producerSnapshot); + testHarness.open(); + testHarness.processElement(45, 7); + testHarness.snapshot(2, 8); + testHarness.processElement(46, 9); + testHarness.notifyOfCompletedCheckpoint(2); + testHarness.processElement(47, 9); + } + } + + // ----------------------------------------------------------------------------------------------------------------- + // shut down a Kafka broker private void failBroker(int brokerId) { KafkaServer toShutDown = null; @@ -602,7 +648,13 @@ private void closeIgnoringProducerFenced(AutoCloseable autoCloseable) throws Exc } private OneInputStreamOperatorTestHarness createTestHarness(String topic) throws Exception { - return createTestHarness(topic, 1, 1, 0, FlinkKafkaProducer.Semantic.EXACTLY_ONCE); + return createTestHarness(topic, FlinkKafkaProducer.Semantic.EXACTLY_ONCE); + } + + private OneInputStreamOperatorTestHarness createTestHarness( + String topic, + FlinkKafkaProducer.Semantic semantic) throws Exception { + return createTestHarness(topic, 1, 1, 0, semantic); } private OneInputStreamOperatorTestHarness createTestHarness( diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java index 03f12b585ae9e..d2735d566eef7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java @@ -320,9 +320,9 @@ public void initializeState(FunctionInitializationContext context) throws Except state = context.getOperatorStateStore().getListState(stateDescriptor); + boolean recoveredUserContext = false; if (context.isRestored()) { LOG.info("{} - restoring state", name()); - for (State operatorState : state.get()) { userContext = operatorState.getContext(); List> recoveredTransactions = operatorState.getPendingCommitTransactions(); @@ -337,11 +337,13 @@ public void initializeState(FunctionInitializationContext context) throws Except if (userContext.isPresent()) { finishRecoveringContext(); + recoveredUserContext = true; } } } + // if in restore we didn't get any userContext or we are initializing from scratch - if (userContext == null) { + if (!recoveredUserContext) { LOG.info("{} - no state to restore", name()); userContext = initializeUserContext(); From 4609341d0d774d6afb5559dbfa41a093fd405ae6 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 6 Nov 2018 13:24:50 +0100 Subject: [PATCH 023/359] [hotfix][tests] Deduplicate the default timeout constants in FlinkKafkaProducerITCase --- .../kafka/FlinkKafkaProducer011ITCase.java | 21 +++++++++---------- .../connectors/kafka/KafkaTestBase.java | 10 ++++++++- .../kafka/FlinkKafkaProducerITCase.java | 21 +++++++++---------- 3 files changed, 29 insertions(+), 23 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index 39a6273318ed7..ca75b1a637baf 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -155,7 +155,7 @@ public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exceptio testHarness2.open(); } - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42)); deleteTestTopic(topic); } catch (Exception ex) { @@ -203,7 +203,7 @@ public void testFlinkKafkaProducer011FailBeforeNotify() throws Exception { testHarness.initializeState(snapshot); testHarness.close(); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43)); deleteTestTopic(topic); } @@ -252,7 +252,7 @@ public void testFlinkKafkaProducer011FailTransactionCoordinatorBeforeNotify() th testHarness2.open(); } - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43)); deleteTestTopic(topic); } @@ -298,7 +298,7 @@ public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { // - aborted transactions with records 44 and 45 // - committed transaction with record 46 // - pending transaction with record 47 - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 46), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 46)); testHarness.close(); deleteTestTopic(topic); @@ -347,7 +347,7 @@ public void testFailAndRecoverSameCheckpointTwice() throws Exception { //now we should have: // - records 42 and 43 in committed transactions // - aborted transactions with records 44 and 45 - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43)); deleteTestTopic(topic); } @@ -399,7 +399,7 @@ public void testScaleDownBeforeFirstCheckpoint() throws Exception { // - records 42, 43, 44 and 45 in aborted transactions // - committed transaction with record 46 // - pending transaction with record 47 - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(46), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(46)); postScaleDownOperator1.close(); // ignore ProducerFencedExceptions, because postScaleDownOperator1 could reuse transactional ids. @@ -468,8 +468,7 @@ public void testScaleUpAfterScalingDown() throws Exception { createProperties(), topic, 0, - IntStream.range(0, parallelism1 + parallelism2 + parallelism3).boxed().collect(Collectors.toList()), - 30_000L); + IntStream.range(0, parallelism1 + parallelism2 + parallelism3).boxed().collect(Collectors.toList())); deleteTestTopic(topic); } @@ -541,7 +540,7 @@ public void testRecoverCommittedTransaction() throws Exception { testHarness.initializeState(checkpoint0); // recover state 0 - producerA recover and commit txn 0 testHarness.close(); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42)); deleteTestTopic(topic); } @@ -572,7 +571,7 @@ public void testRunOutOfProducersInThePool() throws Exception { public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { String topic = "testMigrateFromAtLeastOnceToExactlyOnce"; testRecoverWithChangeSemantics(topic, AT_LEAST_ONCE, EXACTLY_ONCE); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 44, 45), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 44, 45)); deleteTestTopic(topic); } @@ -580,7 +579,7 @@ public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { String topic = "testMigrateFromExactlyOnceToAtLeastOnce"; testRecoverWithChangeSemantics(topic, EXACTLY_ONCE, AT_LEAST_ONCE); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 45, 46, 47), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 45, 46, 47)); deleteTestTopic(topic); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index 1a20d7e82adf4..2d4ac43fa9a71 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -241,6 +241,14 @@ protected void assertAtLeastOnceForTopic( fail(String.format("Expected to contain all of: <%s>, but was: <%s>", expectedElements, actualElements)); } + protected void assertExactlyOnceForTopic( + Properties properties, + String topic, + int partition, + List expectedElements) { + assertExactlyOnceForTopic(properties, topic, partition, expectedElements, 30_000L); + } + /** * We manually handle the timeout instead of using JUnit's timeout to return failure instead of timeout error. * After timeout we assume that there are missing records and there is a bug, not that the test has run out of time. @@ -250,7 +258,7 @@ protected void assertExactlyOnceForTopic( String topic, int partition, List expectedElements, - long timeoutMillis) throws Exception { + long timeoutMillis) { long startMillis = System.currentTimeMillis(); List actualElements = new ArrayList<>(); diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index 6070930ae17d9..29f157ffb4fc3 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -151,7 +151,7 @@ public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exceptio testHarness2.open(); } - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42)); deleteTestTopic(topic); } catch (Exception ex) { @@ -199,7 +199,7 @@ public void testFlinkKafkaProducerFailBeforeNotify() throws Exception { testHarness.initializeState(snapshot); testHarness.close(); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43)); deleteTestTopic(topic); } @@ -248,7 +248,7 @@ public void testFlinkKafkaProducerFailTransactionCoordinatorBeforeNotify() throw testHarness2.open(); } - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43)); deleteTestTopic(topic); } @@ -294,7 +294,7 @@ public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { // - aborted transactions with records 44 and 45 // - committed transaction with record 46 // - pending transaction with record 47 - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 46), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 46)); testHarness.close(); deleteTestTopic(topic); @@ -343,7 +343,7 @@ public void testFailAndRecoverSameCheckpointTwice() throws Exception { //now we should have: // - records 42 and 43 in committed transactions // - aborted transactions with records 44 and 45 - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43)); deleteTestTopic(topic); } @@ -395,7 +395,7 @@ public void testScaleDownBeforeFirstCheckpoint() throws Exception { // - records 42, 43, 44 and 45 in aborted transactions // - committed transaction with record 46 // - pending transaction with record 47 - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(46), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(46)); postScaleDownOperator1.close(); // ignore ProducerFencedExceptions, because postScaleDownOperator1 could reuse transactional ids. @@ -464,8 +464,7 @@ public void testScaleUpAfterScalingDown() throws Exception { createProperties(), topic, 0, - IntStream.range(0, parallelism1 + parallelism2 + parallelism3).boxed().collect(Collectors.toList()), - 30_000L); + IntStream.range(0, parallelism1 + parallelism2 + parallelism3).boxed().collect(Collectors.toList())); deleteTestTopic(topic); } @@ -537,7 +536,7 @@ public void testRecoverCommittedTransaction() throws Exception { testHarness.initializeState(checkpoint0); // recover state 0 - producerA recover and commit txn 0 testHarness.close(); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42)); deleteTestTopic(topic); } @@ -568,7 +567,7 @@ public void testRunOutOfProducersInThePool() throws Exception { public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { String topic = "testMigrateFromAtLeastOnceToExactlyOnce"; testRecoverWithChangeSemantics(topic, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE, FlinkKafkaProducer.Semantic.EXACTLY_ONCE); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 44, 45), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 44, 45)); deleteTestTopic(topic); } @@ -576,7 +575,7 @@ public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { String topic = "testMigrateFromExactlyOnceToAtLeastOnce"; testRecoverWithChangeSemantics(topic, FlinkKafkaProducer.Semantic.EXACTLY_ONCE, FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); - assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 45, 46, 47), 30_000L); + assertExactlyOnceForTopic(createProperties(), topic, 0, Arrays.asList(42, 43, 45, 46, 47)); deleteTestTopic(topic); } From 193faaa00711494e3a171eb1d1c70afebf97520b Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 6 Nov 2018 19:32:42 +0800 Subject: [PATCH 024/359] [FLINK-10704][end-to-end] Fix sql client end to end test failure --- flink-end-to-end-tests/test-scripts/common.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 4f628fc6d1240..ce1407450bed2 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -320,6 +320,7 @@ function check_logs_for_errors { | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/yarn/exceptions/YarnException" \ | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \ | grep -v "org.apache.flink.fs.shaded.hadoop3.org.apache.commons.beanutils.FluentPropertyBeanIntrospector - Error when creating PropertyDescriptor for public final void org.apache.flink.fs.shaded.hadoop3.org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)! Ignoring this property." \ + | grep -v "Error while loading kafka-version.properties :null" \ | grep -ic "error") if [[ ${error_count} -gt 0 ]]; then echo "Found error in log files:" From 98198d4cda78cc815fe6430e2249130efb102b61 Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Sat, 20 Oct 2018 15:40:06 +0800 Subject: [PATCH 025/359] [FLINK-10463][table] Null literal cannot be properly parsed in Java Table API function call --- docs/dev/table/functions.md | 6 ++-- docs/dev/table/tableApi.md | 28 ++++++++++++++----- .../table/expressions/ExpressionParser.scala | 2 +- .../expressions/ScalarFunctionsTest.scala | 7 +++++ 4 files changed, 32 insertions(+), 11 deletions(-) diff --git a/docs/dev/table/functions.md b/docs/dev/table/functions.md index e47b31772b38d..e652c304e932c 100644 --- a/docs/dev/table/functions.md +++ b/docs/dev/table/functions.md @@ -4186,7 +4186,7 @@ CAST(value AS type)

Returns a new value being cast to type type. See the supported types here.

-

E.g., CAST('42' AS INT) returns 42.

+

E.g., CAST('42' AS INT) returns 42; CAST(NULL AS VARCHAR) returns NULL of type VARCHAR.

@@ -4211,7 +4211,7 @@ ANY.cast(TYPE)

Returns a new ANY being cast to type TYPE. See the supported types here.

-

E.g., '42'.cast(INT) returns 42.

+

E.g., '42'.cast(INT) returns 42; Null(STRING) returns NULL of type STRING.

@@ -4236,7 +4236,7 @@ ANY.cast(TYPE)

Returns a new ANY being cast to type TYPE. See the supported types here.

-

E.g., "42".cast(Types.INT) returns 42.

+

E.g., "42".cast(Types.INT) returns 42; Null(Types.STRING) returns NULL of type STRING.

diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index 76bd5b28ef285..b17753c16e8b4 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -1738,11 +1738,13 @@ term = product , [ ( "+" | "-" ) , product ] ; product = unary , [ ( "*" | "/" | "%") , unary ] ; -unary = [ "!" | "-" ] , composite ; +unary = [ "!" | "-" | "+" ] , composite ; -composite = over | nullLiteral | suffixed | atom ; +composite = over | suffixed | nullLiteral | prefixed | atom ; -suffixed = interval | cast | as | if | functionCall ; +suffixed = interval | suffixAs | suffixCast | suffixIf | suffixDistinct | suffixFunctionCall ; + +prefixed = prefixAs | prefixCast | prefixIf | prefixDistinct | prefixFunctionCall ; interval = timeInterval | rowInterval ; @@ -1750,15 +1752,27 @@ timeInterval = composite , "." , ("year" | "years" | "quarter" | "quarters" | "m rowInterval = composite , "." , "rows" ; -cast = composite , ".cast(" , dataType , ")" ; +suffixCast = composite , ".cast(" , dataType , ")" ; + +prefixCast = "cast(" , expression , dataType , ")" ; dataType = "BYTE" | "SHORT" | "INT" | "LONG" | "FLOAT" | "DOUBLE" | "BOOLEAN" | "STRING" | "DECIMAL" | "SQL_DATE" | "SQL_TIME" | "SQL_TIMESTAMP" | "INTERVAL_MONTHS" | "INTERVAL_MILLIS" | ( "MAP" , "(" , dataType , "," , dataType , ")" ) | ( "PRIMITIVE_ARRAY" , "(" , dataType , ")" ) | ( "OBJECT_ARRAY" , "(" , dataType , ")" ) ; -as = composite , ".as(" , fieldReference , ")" ; +suffixAs = composite , ".as(" , fieldReference , ")" ; + +prefixAs = "as(" , expression, fieldReference , ")" ; + +suffixIf = composite , ".?(" , expression , "," , expression , ")" ; + +prefixIf = "?(" , expression , "," , expression , "," , expression , ")" ; + +suffixDistinct = composite , "distinct.()" ; + +prefixDistinct = functionIdentifier , ".distinct" , [ "(" , [ expression , { "," , expression } ] , ")" ] ; -if = composite , ".?(" , expression , "," , expression , ")" ; +suffixFunctionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ; -functionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ; +prefixFunctionCall = functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ; atom = ( "(" , expression , ")" ) | literal | fieldReference ; diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala index 0b84f9490a135..7fd9309b5dbb1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala @@ -455,7 +455,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { // suffix/prefix composite - lazy val composite: PackratParser[Expression] = over | nullLiteral | suffixed | prefixed | atom | + lazy val composite: PackratParser[Expression] = over | suffixed | nullLiteral | prefixed | atom | failure("Composite expression expected.") // unary ops diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala index 42513c2a96fc1..60cdda1f4e40c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala @@ -645,6 +645,13 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { "'foobar'.regexpReplace('oo|ar', f33)", "REGEXP_REPLACE('foobar', 'oo|ar', f33)", "null") + + // This test was added for the null literal problem in string expression parsing (FLINK-10463). + testAllApis( + Null(Types.STRING).regexpReplace("oo|ar", 'f33), + "Null(STRING).regexpReplace('oo|ar', f33)", + "REGEXP_REPLACE(CAST(NULL AS VARCHAR), 'oo|ar', f33)", + "null") } @Test From 9db96301f22403fe84bb279df1f348dff7a2974f Mon Sep 17 00:00:00 2001 From: Tzu-Li Chen Date: Wed, 7 Nov 2018 21:41:44 +0800 Subject: [PATCH 026/359] [FLINK-10797][docs] Fix broken ide_setup links --- README.md | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 7966ecd3d7659..64868e1869ccc 100644 --- a/README.md +++ b/README.md @@ -100,7 +100,7 @@ The IntelliJ IDE supports Maven out of the box and offers a plugin for Scala dev * IntelliJ download: [https://www.jetbrains.com/idea/](https://www.jetbrains.com/idea/) * IntelliJ Scala Plugin: [http://plugins.jetbrains.com/plugin/?id=1347](http://plugins.jetbrains.com/plugin/?id=1347) -Check out our [Setting up IntelliJ](https://github.com/apache/flink/blob/master/docs/internals/ide_setup.md#intellij-idea) guide for details. +Check out our [Setting up IntelliJ](https://ci.apache.org/projects/flink/flink-docs-master/flinkDev/ide_setup.html#intellij-idea) guide for details. ### Eclipse Scala IDE diff --git a/pom.xml b/pom.xml index 055e64896e55a..7f176facc5890 100644 --- a/pom.xml +++ b/pom.xml @@ -1556,7 +1556,7 @@ under the License. com.puppycrawl.tools checkstyle - + 8.9 From 7d980fdb6f126f7bd46904c610c0da3dc232e657 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 7 Nov 2018 11:12:02 +0100 Subject: [PATCH 027/359] [FLINK-10711] [e2e] Allow basic error handling with bash This commit modifies the test infrastructure to allow bash's basic error handling mechanism with set -e. Many tests are not ready for a globally defined strict error handling. For now, at least newly developed tests should consider this flag. If a test causes an error, a test fails with "[FAIL] Test script contains errors". See the end-to-end test README for more information about how to develop tests in the future. This closes #7023. --- flink-end-to-end-tests/README.md | 2 ++ flink-end-to-end-tests/test-scripts/common.sh | 25 ++++++++++++++----- .../test-scripts/elasticsearch-common.sh | 2 -- .../test-scripts/kafka-common.sh | 4 --- .../test-scripts/test-runner-common.sh | 22 ++++++++++++++-- .../test-scripts/test_batch_allround.sh | 2 ++ .../test-scripts/test_cli.sh | 2 ++ .../test-scripts/test_streaming_kafka.sh | 4 +-- .../test-scripts/test_streaming_kafka010.sh | 2 ++ .../test_streaming_kafka_common.sh | 4 --- 10 files changed, 48 insertions(+), 21 deletions(-) diff --git a/flink-end-to-end-tests/README.md b/flink-end-to-end-tests/README.md index 257298e3cceec..0afc96f227654 100644 --- a/flink-end-to-end-tests/README.md +++ b/flink-end-to-end-tests/README.md @@ -47,6 +47,8 @@ In order to add a new test case you need add it to either `test-scripts/run-nigh _Note: If you want to parameterize your tests please do so by adding multiple test cases with parameters as arguments to the nightly / pre-commit test suites. This allows the test runner to do a cleanup in between each individual test and also to fail those tests individually._ +_Note: While developing a new test case make sure to enable bash's error handling in `test-scripts/common.sh` by uncommenting `set -Eexuo pipefail` and commenting the current default `set` call. Once your test is implemented properly, add `set -Eeuo pipefail` on the very top of your test script (before any `common` script)._ + ### Passing your test A test is considered to have passed if it: - has exit code 0 diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index ce1407450bed2..113bb3a8c915d 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +# Enable this line when developing a new end-to-end test +#set -Eexuo pipefail set -o pipefail if [[ -z $FLINK_DIR ]]; then @@ -304,6 +306,7 @@ function start_and_wait_for_tm { } function check_logs_for_errors { + echo "Checking for errors..." error_count=$(grep -rv "GroupCoordinatorNotAvailableException" $FLINK_DIR/log \ | grep -v "RetriableCommitFailedException" \ | grep -v "NoAvailableBrokersException" \ @@ -321,15 +324,18 @@ function check_logs_for_errors { | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \ | grep -v "org.apache.flink.fs.shaded.hadoop3.org.apache.commons.beanutils.FluentPropertyBeanIntrospector - Error when creating PropertyDescriptor for public final void org.apache.flink.fs.shaded.hadoop3.org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)! Ignoring this property." \ | grep -v "Error while loading kafka-version.properties :null" \ - | grep -ic "error") + | grep -ic "error" || true) if [[ ${error_count} -gt 0 ]]; then echo "Found error in log files:" cat $FLINK_DIR/log/* EXIT_CODE=1 + else + echo "No errors in log files." fi } function check_logs_for_exceptions { + echo "Checking for exceptions..." exception_count=$(grep -rv "GroupCoordinatorNotAvailableException" $FLINK_DIR/log \ | grep -v "RetriableCommitFailedException" \ | grep -v "NoAvailableBrokersException" \ @@ -349,19 +355,24 @@ function check_logs_for_exceptions { | grep -v "java.io.InvalidClassException: org.apache.flink.formats.avro.typeutils.AvroSerializer" \ | grep -v "Caused by: java.lang.Exception: JobManager is shutting down" \ | grep -v "java.lang.Exception: Artificial failure" \ - | grep -ic "exception") + | grep -ic "exception" || true) if [[ ${exception_count} -gt 0 ]]; then echo "Found exception in log files:" cat $FLINK_DIR/log/* EXIT_CODE=1 + else + echo "No exceptions in log files." fi } function check_logs_for_non_empty_out_files { + echo "Checking for non-empty .out files..." if grep -ri "." $FLINK_DIR/log/*.out > /dev/null; then echo "Found non-empty .out files:" cat $FLINK_DIR/log/*.out EXIT_CODE=1 + else + echo "No non-empty .out files." fi } @@ -375,7 +386,9 @@ function stop_cluster { "$FLINK_DIR"/bin/stop-cluster.sh # stop zookeeper only if there are processes running - if ! [ "`jps | grep 'FlinkZooKeeperQuorumPeer' | wc -l`" = "0" ]; then + zookeeper_process_count=$(jps | grep -c 'FlinkZooKeeperQuorumPeer' || true) + if [[ ${zookeeper_process_count} -gt 0 ]]; then + echo "Stopping zookeeper..." "$FLINK_DIR"/bin/zookeeper.sh stop fi } @@ -490,9 +503,9 @@ function tm_kill_all { # Kills all processes that match the given name. function kill_all { - local pid=`jps | grep -E "${1}" | cut -d " " -f 1` - kill ${pid} 2> /dev/null - wait ${pid} 2> /dev/null + local pid=`jps | grep -E "${1}" | cut -d " " -f 1 || true` + kill ${pid} 2> /dev/null || true + wait ${pid} 2> /dev/null || true } function kill_random_taskmanager { diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 834e84528b3d3..3d64b31b89771 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -17,8 +17,6 @@ # limitations under the License. ################################################################################ -set -o pipefail - if [[ -z $TEST_DATA_DIR ]]; then echo "Must run common.sh before elasticsearch-common.sh." exit 1 diff --git a/flink-end-to-end-tests/test-scripts/kafka-common.sh b/flink-end-to-end-tests/test-scripts/kafka-common.sh index dedfe5208b1ae..5f8f65ec2515d 100644 --- a/flink-end-to-end-tests/test-scripts/kafka-common.sh +++ b/flink-end-to-end-tests/test-scripts/kafka-common.sh @@ -17,10 +17,6 @@ # limitations under the License. ################################################################################ -set -e -set -u -set -o pipefail - if [[ -z $TEST_DATA_DIR ]]; then echo "Must run common.sh before kafka-common.sh." exit 1 diff --git a/flink-end-to-end-tests/test-scripts/test-runner-common.sh b/flink-end-to-end-tests/test-scripts/test-runner-common.sh index 0e56d2ac250d4..44ec80964bbbc 100644 --- a/flink-end-to-end-tests/test-scripts/test-runner-common.sh +++ b/flink-end-to-end-tests/test-scripts/test-runner-common.sh @@ -29,7 +29,7 @@ source "${END_TO_END_DIR}"/test-scripts/common.sh function run_test { local description="$1" local command="$2" - local skip_check_exceptions="$3" + local skip_check_exceptions=${3:-} printf "\n==============================================================================\n" printf "Running '${description}'\n" @@ -41,14 +41,32 @@ function run_test { backup_config start_timer + + function test_error() { + echo "[FAIL] Test script contains errors." + post_test_validation 1 "$description" "$skip_check_exceptions" + } + trap 'test_error' ERR + ${command} exit_code="$?" - time_elapsed=$(end_timer) + post_test_validation ${exit_code} "$description" "$skip_check_exceptions" +} + +# Validates the test result and exit code after its execution. +function post_test_validation { + local exit_code="$1" + local description="$2" + local skip_check_exceptions="$3" + + local time_elapsed=$(end_timer) if [[ "${skip_check_exceptions}" != "skip_check_exceptions" ]]; then check_logs_for_errors check_logs_for_exceptions check_logs_for_non_empty_out_files + else + echo "Checking of logs skipped." fi # Investigate exit_code for failures of test executable as well as EXIT_CODE for failures of the test. diff --git a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh index f8f591f7dc0be..2bdda8195d87d 100755 --- a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh +++ b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +set -Eeuo pipefail + source "$(dirname "$0")"/common.sh TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-dataset-allround-test/target/DataSetAllroundTestProgram.jar diff --git a/flink-end-to-end-tests/test-scripts/test_cli.sh b/flink-end-to-end-tests/test-scripts/test_cli.sh index 9e42b3b645758..b9d285b635204 100755 --- a/flink-end-to-end-tests/test-scripts/test_cli.sh +++ b/flink-end-to-end-tests/test-scripts/test_cli.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +set -Eeuo pipefail + source "$(dirname "$0")"/common.sh TEST_PROGRAM_JAR=$END_TO_END_DIR/flink-cli-test/target/PeriodicStreamingJob.jar diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh index c5cdfde3dc0e5..044d223798811 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh @@ -17,9 +17,7 @@ # limitations under the License. ################################################################################ -set -e -set -u -set -o pipefail +set -Eeuo pipefail source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/kafka-common.sh 2.0.0 5.0.0 5.0 diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh index ecd651448a975..a55b0bc33c68b 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh @@ -17,6 +17,8 @@ # limitations under the License. ################################################################################ +set -Eeuo pipefail + source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/kafka-common.sh 0.10.2.0 3.2.0 3.2 diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh index ff3adc158c7fd..0583638b3bcf9 100644 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh @@ -17,10 +17,6 @@ # limitations under the License. ################################################################################ -set -e -set -u -set -o pipefail - KAFKA_EXAMPLE_JAR="$1" setup_kafka_dist From 1a16202b6eb9352192092e0c36a2e7d2aabc2ae9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 6 Nov 2018 17:01:30 +0100 Subject: [PATCH 028/359] [FLINK-10805] [e2e] Fix failing end-to-end tests Fix test_confluent_schema_registry.sh and test_sql_client.sh end-to-end tests which failed because of missing arguments for kafka-common.sh and the newly introduce set -e flag. This closes #7034. --- flink-end-to-end-tests/test-scripts/common.sh | 2 +- .../test-scripts/elasticsearch-common.sh | 6 +++--- .../test_confluent_schema_registry.sh | 4 +++- .../test-scripts/test_sql_client.sh | 19 +++++++++++++------ 4 files changed, 20 insertions(+), 11 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 113bb3a8c915d..36cc0df142ce1 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -432,7 +432,7 @@ function wait_job_terminal_state { echo "Waiting for job ($job) to reach terminal state $terminal_state ..." while : ; do - N=$(grep -o "Job $job reached globally terminal state $terminal_state" $FLINK_DIR/log/*standalonesession*.log | tail -1) + N=$(grep -o "Job $job reached globally terminal state $terminal_state" $FLINK_DIR/log/*standalonesession*.log | tail -1 || true) if [[ -z $N ]]; then sleep 1 diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 3d64b31b89771..16689c8a3f880 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -43,7 +43,7 @@ function wait_elasticsearch_working { echo "Waiting for Elasticsearch node to work..." for ((i=1;i<=60;i++)); do - curl -XGET 'http://localhost:9200' + curl -XGET 'http://localhost:9200' || true # make sure the elasticsearch node is actually working if [ $? -ne 0 ]; then @@ -67,7 +67,7 @@ function verify_result_line_number { fi while : ; do - curl "localhost:9200/${index}/_search?q=*&pretty&size=21" > $TEST_DATA_DIR/output + curl "localhost:9200/${index}/_search?q=*&pretty&size=21" > $TEST_DATA_DIR/output || true if [ -n "$(grep "\"total\" : $numRecords" $TEST_DATA_DIR/output)" ]; then echo "Elasticsearch end to end test pass." @@ -86,7 +86,7 @@ function verify_result_hash { local hash=$4 while : ; do - curl "localhost:9200/${index}/_search?q=*&pretty" > $TEST_DATA_DIR/es_output + curl "localhost:9200/${index}/_search?q=*&pretty" > $TEST_DATA_DIR/es_output || true if [ -n "$(grep "\"total\" : $numRecords" $TEST_DATA_DIR/es_output)" ]; then break diff --git a/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh b/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh index 323234fa683d0..b1500a0a96a62 100755 --- a/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh +++ b/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh @@ -17,8 +17,10 @@ # limitations under the License. ################################################################################ +set -Eeuo pipefail + source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh +source "$(dirname "$0")"/kafka-common.sh 0.10.2.0 3.2.0 3.2 function verify_output { local expected=$(printf $1) diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index ce3d194487766..313ee4436294d 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -17,8 +17,10 @@ # limitations under the License. ################################################################################ +set -Eeuo pipefail + source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh +source "$(dirname "$0")"/kafka-common.sh 0.10.2.0 3.2.0 3.2 source "$(dirname "$0")"/elasticsearch-common.sh SQL_TOOLBOX_JAR=$END_TO_END_DIR/flink-sql-client-test/target/SqlToolbox.jar @@ -306,7 +308,7 @@ EOF echo "Executing SQL: Kafka JSON -> Kafka Avro" -read -r -d '' SQL_STATEMENT_1 << EOF +SQL_STATEMENT_1=$(cat << EOF INSERT INTO AvroBothTable SELECT CAST(TUMBLE_START(rowtime, INTERVAL '1' HOUR) AS VARCHAR) AS event_timestamp, @@ -320,6 +322,7 @@ INSERT INTO AvroBothTable event.message, TUMBLE(rowtime, INTERVAL '1' HOUR) EOF +) echo "$SQL_STATEMENT_1" @@ -331,11 +334,12 @@ $FLINK_DIR/bin/sql-client.sh embedded \ echo "Executing SQL: Kafka Avro -> Filesystem CSV" -read -r -d '' SQL_STATEMENT_2 << EOF +SQL_STATEMENT_2=$(cat << EOF INSERT INTO CsvSinkTable SELECT AvroBothTable.*, RegReplace('Test constant folding.', 'Test', 'Success') AS constant FROM AvroBothTable EOF +) echo "$SQL_STATEMENT_2" @@ -360,13 +364,14 @@ check_result_hash "SQL Client Kafka" $RESULT "0a1bf8bf716069b7269f575f87a802c0" echo "Executing SQL: Values -> Elasticsearch (upsert)" -read -r -d '' SQL_STATEMENT_3 << EOF +SQL_STATEMENT_3=$(cat << EOF INSERT INTO ElasticsearchUpsertSinkTable SELECT user_id, user_name, COUNT(*) AS user_count FROM (VALUES (1, 'Bob'), (22, 'Alice'), (42, 'Greg'), (42, 'Greg'), (42, 'Greg'), (1, 'Bob')) AS UserCountTable(user_id, user_name) GROUP BY user_id, user_name EOF +) JOB_ID=$($FLINK_DIR/bin/sql-client.sh embedded \ --library $SQL_JARS_DIR \ @@ -380,7 +385,7 @@ verify_result_hash "SQL Client Elasticsearch Upsert" "$ELASTICSEARCH_INDEX" 3 "9 echo "Executing SQL: Values -> Elasticsearch (append, no key)" -read -r -d '' SQL_STATEMENT_4 << EOF +SQL_STATEMENT_4=$(cat << EOF INSERT INTO ElasticsearchAppendSinkTable SELECT * FROM ( @@ -393,6 +398,7 @@ INSERT INTO ElasticsearchAppendSinkTable (1, 'Bob', CAST(0 AS BIGINT))) AS UserCountTable(user_id, user_name, user_count) EOF +) JOB_ID=$($FLINK_DIR/bin/sql-client.sh embedded \ --library $SQL_JARS_DIR \ @@ -407,7 +413,7 @@ verify_result_line_number 9 "$ELASTICSEARCH_INDEX" echo "Executing SQL: Match recognize -> Elasticsearch" -read -r -d '' SQL_STATEMENT_5 << EOF +SQL_STATEMENT_5=$(cat << EOF INSERT INTO ElasticsearchAppendSinkTable SELECT 1 as user_id, T.userName as user_name, cast(1 as BIGINT) as user_count FROM ( @@ -423,6 +429,7 @@ INSERT INTO ElasticsearchAppendSinkTable A as user = 'Alice' ) T EOF +) JOB_ID=$($FLINK_DIR/bin/sql-client.sh embedded \ --library $SQL_JARS_DIR \ From 600869e1f26c5b43e68e66023c95fc1ea0267c37 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Nov 2018 09:44:36 +0100 Subject: [PATCH 029/359] [FLINK-10750][tests] Harden SocketClientSinkTest The tests attempts to re-bind a server to a specific port. If the port is taken the test will fail. The test is now skipped instead if this happens. This closes #7040. --- .../api/functions/sink/SocketClientSinkTest.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java index b3c4ee93ede60..7c9863ef07ff5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java @@ -24,11 +24,13 @@ import org.apache.flink.util.TestLogger; import org.apache.commons.io.IOUtils; +import org.junit.AssumptionViolatedException; import org.junit.Test; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; +import java.net.BindException; import java.net.ServerSocket; import java.net.Socket; import java.util.concurrent.Callable; @@ -279,7 +281,12 @@ public Void call() throws Exception { retryLatch.countDown(); // Restart the server - serverSocket[0] = new ServerSocket(port); + try { + serverSocket[0] = new ServerSocket(port); + } catch (BindException be) { + // some other process may be using this port now + throw new AssumptionViolatedException("Could not bind server to previous port.", be); + } Socket socket = serverSocket[0].accept(); BufferedReader reader = new BufferedReader(new InputStreamReader( From ef4394188988568f805f40848f420173f36a4b12 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Nov 2018 14:20:50 +0100 Subject: [PATCH 030/359] [FLINK-10812][build] Skip javadoc plugin for e2e-test-utils --- flink-end-to-end-tests/flink-e2e-test-utils/pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml b/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml index 219d6626ef373..fd4942f2b9bef 100644 --- a/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml +++ b/flink-end-to-end-tests/flink-e2e-test-utils/pom.xml @@ -65,6 +65,15 @@ under the License. + + + org.apache.maven.plugins + maven-javadoc-plugin + + + true + +
From 34938c4c44282d4b5e3764dd8ba7ee5b767a8a96 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 7 Nov 2018 17:05:50 +0100 Subject: [PATCH 031/359] [hotfix] Add RocksDB config options to documentation --- docs/ops/config.md | 4 ++++ docs/ops/state/state_backends.md | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/docs/ops/config.md b/docs/ops/config.md index 267b670d15a80..79e768c394603 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -154,6 +154,10 @@ The configuration keys in this section are independent of the used resource mana {% include generated/checkpointing_configuration.html %} +### RocksDB State Backend + +{% include generated/rocks_db_configuration.html %} + ### Queryable State {% include generated/queryable_state_configuration.html %} diff --git a/docs/ops/state/state_backends.md b/docs/ops/state/state_backends.md index b95d2b7e4dbe6..e4fe5bda06304 100644 --- a/docs/ops/state/state_backends.md +++ b/docs/ops/state/state_backends.md @@ -171,4 +171,8 @@ state.backend: filesystem state.checkpoints.dir: hdfs://namenode:40010/flink/checkpoints {% endhighlight %} +#### RocksDB State Backend Config Options + +{% include generated/rocks_db_configuration.html %} + {% top %} From a990f844cf724cd85fdde52e388dbb5381b3a343 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Nov 2018 13:07:46 +0100 Subject: [PATCH 032/359] [FLINK-10811][hcatalog] Add scala suffix --- flink-connectors/flink-hcatalog/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml index 3b5f78ddea9bb..74e715b4601ea 100644 --- a/flink-connectors/flink-hcatalog/pom.xml +++ b/flink-connectors/flink-hcatalog/pom.xml @@ -29,7 +29,7 @@ under the License. .. - flink-hcatalog + flink-hcatalog_${scala.binary.version} flink-hcatalog jar From b975ea964c6e33d9e60e6471af3c9ed6f0a9f901 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 7 Nov 2018 18:06:34 +0100 Subject: [PATCH 033/359] [FLINK-10816][cep] Fix LockableTypeSerializer.duplicate() to consider deep duplication of element serializer This closes #7049. --- .../flink/cep/nfa/sharedbuffer/Lockable.java | 14 +- .../LockableTypeSerializerTest.java | 53 +++++++ .../state/heap/CopyOnWriteStateTableTest.java | 103 -------------- .../state/heap/TestDuplicateSerializer.java | 129 ++++++++++++++++++ 4 files changed, 193 insertions(+), 106 deletions(-) create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/TestDuplicateSerializer.java diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java index a2eedaf138c6c..4ba4592fd9927 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java @@ -18,6 +18,7 @@ package org.apache.flink.cep.nfa.sharedbuffer; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -109,8 +110,10 @@ public boolean isImmutableType() { } @Override - public TypeSerializer> duplicate() { - return new LockableTypeSerializer<>(elementSerializer); + public LockableTypeSerializer duplicate() { + TypeSerializer elementSerializerCopy = elementSerializer.duplicate(); + return elementSerializerCopy == elementSerializer ? + this : new LockableTypeSerializer<>(elementSerializerCopy); } @Override @@ -120,7 +123,7 @@ public Lockable createInstance() { @Override public Lockable copy(Lockable from) { - return new Lockable(elementSerializer.copy(from.element), from.refCounter); + return new Lockable<>(elementSerializer.copy(from.element), from.refCounter); } @Override @@ -219,5 +222,10 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfig : CompatibilityResult.compatible(); } } + + @VisibleForTesting + TypeSerializer getElementSerializer() { + return elementSerializer; + } } } diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerTest.java new file mode 100644 index 0000000000000..a7aeb69ff6adc --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerTest.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.cep.nfa.sharedbuffer; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.runtime.state.heap.TestDuplicateSerializer; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link org.apache.flink.cep.nfa.sharedbuffer.Lockable.LockableTypeSerializer}. + */ +public class LockableTypeSerializerTest { + + /** + * This tests that {@link Lockable.LockableTypeSerializer#duplicate()} works as expected. + */ + @Test + public void testDuplicate() { + IntSerializer nonDuplicatingInnerSerializer = IntSerializer.INSTANCE; + Assert.assertSame(nonDuplicatingInnerSerializer, nonDuplicatingInnerSerializer.duplicate()); + Lockable.LockableTypeSerializer candidateTestShallowDuplicate = + new Lockable.LockableTypeSerializer<>(nonDuplicatingInnerSerializer); + Assert.assertSame(candidateTestShallowDuplicate, candidateTestShallowDuplicate.duplicate()); + + TestDuplicateSerializer duplicatingInnerSerializer = new TestDuplicateSerializer(); + Assert.assertNotSame(duplicatingInnerSerializer, duplicatingInnerSerializer.duplicate()); + + Lockable.LockableTypeSerializer candidateTestDeepDuplicate = + new Lockable.LockableTypeSerializer<>(duplicatingInnerSerializer); + + Lockable.LockableTypeSerializer deepDuplicate = candidateTestDeepDuplicate.duplicate(); + Assert.assertNotSame(candidateTestDeepDuplicate, deepDuplicate); + Assert.assertNotSame(candidateTestDeepDuplicate.getElementSerializer(), deepDuplicate.getElementSerializer()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java index 54cd9c9c83730..ba79004bc7938 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java @@ -19,15 +19,11 @@ package org.apache.flink.runtime.state.heap; import org.apache.flink.api.common.state.StateDescriptor; -import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.state.ArrayListSerializer; import org.apache.flink.runtime.state.KeyGroupRange; @@ -564,103 +560,4 @@ public TypeSerializer getKeySerializer() { } } - /** - * Serializer that can be disabled. Duplicates are still enabled, so we can check that - * serializers are duplicated. - */ - static class TestDuplicateSerializer extends TypeSerializer { - - private static final long serialVersionUID = 1L; - - private static final Integer ZERO = 0; - - private boolean disabled; - - public TestDuplicateSerializer() { - this.disabled = false; - } - - @Override - public boolean isImmutableType() { - return true; - } - - @Override - public TypeSerializer duplicate() { - return new TestDuplicateSerializer(); - } - - @Override - public Integer createInstance() { - return ZERO; - } - - @Override - public Integer copy(Integer from) { - return from; - } - - @Override - public Integer copy(Integer from, Integer reuse) { - return from; - } - - @Override - public int getLength() { - return 4; - } - - @Override - public void serialize(Integer record, DataOutputView target) throws IOException { - Assert.assertFalse(disabled); - target.writeInt(record); - } - - @Override - public Integer deserialize(DataInputView source) throws IOException { - Assert.assertFalse(disabled); - return source.readInt(); - } - - @Override - public Integer deserialize(Integer reuse, DataInputView source) throws IOException { - Assert.assertFalse(disabled); - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - Assert.assertFalse(disabled); - target.writeInt(source.readInt()); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof TestDuplicateSerializer; - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof TestDuplicateSerializer; - } - - @Override - public int hashCode() { - return getClass().hashCode(); - } - - public void disable() { - this.disabled = true; - } - - @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { - throw new UnsupportedOperationException(); - } - - @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - throw new UnsupportedOperationException(); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/TestDuplicateSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/TestDuplicateSerializer.java new file mode 100644 index 0000000000000..83e249be8f805 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/TestDuplicateSerializer.java @@ -0,0 +1,129 @@ +/* + * 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.state.heap; + +import org.apache.flink.api.common.typeutils.CompatibilityResult; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import org.junit.Assert; + +import java.io.IOException; + +/** + * This serializer can be used to test serializer duplication. The serializer that can be disabled. Duplicates are still + * enabled, so we can check that serializers are duplicated. + */ +public class TestDuplicateSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + private static final Integer ZERO = 0; + + private boolean disabled; + + public TestDuplicateSerializer() { + this.disabled = false; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public TypeSerializer duplicate() { + return new TestDuplicateSerializer(); + } + + @Override + public Integer createInstance() { + return ZERO; + } + + @Override + public Integer copy(Integer from) { + return from; + } + + @Override + public Integer copy(Integer from, Integer reuse) { + return from; + } + + @Override + public int getLength() { + return 4; + } + + @Override + public void serialize(Integer record, DataOutputView target) throws IOException { + Assert.assertFalse(disabled); + target.writeInt(record); + } + + @Override + public Integer deserialize(DataInputView source) throws IOException { + Assert.assertFalse(disabled); + return source.readInt(); + } + + @Override + public Integer deserialize(Integer reuse, DataInputView source) throws IOException { + Assert.assertFalse(disabled); + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + Assert.assertFalse(disabled); + target.writeInt(source.readInt()); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof TestDuplicateSerializer; + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof TestDuplicateSerializer; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + public void disable() { + this.disabled = true; + } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException(); + } +} From df81b9d70d1dd7fd2f0dc86fa168f058991b0e33 Mon Sep 17 00:00:00 2001 From: jrthe42 Date: Sat, 3 Nov 2018 21:24:57 +0800 Subject: [PATCH 034/359] [FLINK-10770] [table] Ensure calling open/close of generated functions This closes #7014. --- .../flink/table/runtime/CRowMapRunner.scala | 7 ++++ .../runtime/CRowOutputProcessRunner.scala | 7 ++++ .../flink/table/runtime/FlatJoinRunner.scala | 7 ++++ .../flink/table/runtime/MapRunner.scala | 7 ++++ .../table/runtime/MapSideJoinRunner.scala | 7 ++++ .../OutputRowtimeProcessFunction.scala | 3 ++ .../table/runtime/join/NonWindowJoin.scala | 11 ++++-- .../join/TemporalProcessTimeJoin.scala | 10 ++++++ .../runtime/join/TemporalRowtimeJoin.scala | 34 ++++++++++++------- .../runtime/join/TimeBoundedStreamJoin.scala | 7 ++++ .../match/IterativeConditionRunner.scala | 1 + .../match/PatternSelectFunctionRunner.scala | 1 + .../runtime/outerJoinGroupReduceRunners.scala | 7 ++++ .../table/runtime/outerJoinRunners.scala | 7 ++++ .../utils/userDefinedScalarFunctions.scala | 3 ++ .../runtime/batch/table/JoinITCase.scala | 8 ++++- .../runtime/stream/table/JoinITCase.scala | 8 ++++- 17 files changed, 119 insertions(+), 16 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala index 54bac601bcc10..cb451b92cd7c0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable @@ -46,6 +47,8 @@ class CRowMapRunner[OUT]( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating MapFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) } override def map(in: CRow): OUT = { @@ -53,4 +56,8 @@ class CRowMapRunner[OUT]( } override def getProducedType: TypeInformation[OUT] = returnType + + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala index 600b8987a28f4..ebef577ff8358 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration @@ -49,6 +50,8 @@ class CRowOutputProcessRunner( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating ProcessFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) this.cRowWrapper = new CRowWrappingCollector() this.cRowWrapper.setChange(true) @@ -68,4 +71,8 @@ class CRowOutputProcessRunner( } override def getProducedType: TypeInformation[CRow] = returnType + + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala index 0bf65694367f9..26ed962e01a68 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{FlatJoinFunction, RichFlatJoinFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable @@ -42,10 +43,16 @@ class FlatJoinRunner[IN1, IN2, OUT]( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating FlatJoinFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) } override def join(first: IN1, second: IN2, out: Collector[OUT]): Unit = function.join(first, second, out) override def getProducedType: TypeInformation[OUT] = returnType + + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala index 00d18ecc00794..7f56ffe72cc62 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable @@ -41,10 +42,16 @@ class MapRunner[IN, OUT]( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating MapFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) } override def map(in: IN): OUT = function.map(in) override def getProducedType: TypeInformation[OUT] = returnType + + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala index 5f5a2cc4a19a3..f15aaa6432ee3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{FlatJoinFunction, RichFlatMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable @@ -43,6 +44,8 @@ abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT]( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating FlatJoinFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) broadcastSet = retrieveBroadcastSet } @@ -56,4 +59,8 @@ abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT]( } override def getProducedType: TypeInformation[OUT] = returnType + + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala index 3eaeea308f915..c964a4b7165ae 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala @@ -55,4 +55,7 @@ class OutputRowtimeProcessFunction[OUT]( out.collect(function.map(in)) } + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala index 0fe2e39c3a718..c59f4c2f44b04 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.join import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, ValueState, ValueStateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} @@ -77,14 +78,16 @@ abstract class NonWindowJoin( protected var curProcessTime: Long = _ override def open(parameters: Configuration): Unit = { - LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " + - s"Code:\n$genJoinFuncCode") + LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n Code:\n$genJoinFuncCode") val clazz = compile( getRuntimeContext.getUserCodeClassLoader, genJoinFuncName, genJoinFuncCode) + LOG.debug("Instantiating JoinFunction.") joinFunction = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(joinFunction, getRuntimeContext) + FunctionUtils.openFunction(joinFunction, parameters) // initialize left and right state, the first element of tuple2 indicates how many rows of // this row, while the second element represents the expired time of this row. @@ -291,4 +294,8 @@ abstract class NonWindowJoin( joinFunction.join(otherSideRow, inputRow, cRowWrapper) } } + + override def close(): Unit = { + FunctionUtils.closeFunction(joinFunction) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala index 175af91e9e47d..7a8bb61b1cfce 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala @@ -18,8 +18,10 @@ package org.apache.flink.table.runtime.join import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, TimestampedCollector, TwoInputStreamOperator} import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.table.api.StreamQueryConfig @@ -51,12 +53,16 @@ class TemporalProcessTimeJoin( protected var joinFunction: FlatJoinFunction[Row, Row, Row] = _ override def open(): Unit = { + LOG.debug(s"Compiling FlatJoinFunction: $genJoinFuncName \n\n Code:\n$genJoinFuncCode") val clazz = compile( getRuntimeContext.getUserCodeClassLoader, genJoinFuncName, genJoinFuncCode) + LOG.debug("Instantiating FlatJoinFunction.") joinFunction = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(joinFunction, getRuntimeContext) + FunctionUtils.openFunction(joinFunction, new Configuration()) val rightStateDescriptor = new ValueStateDescriptor[Row]("right", rightType) rightState = getRuntimeContext.getState(rightStateDescriptor) @@ -86,4 +92,8 @@ class TemporalProcessTimeJoin( rightState.clear() } } + + override def close(): Unit = { + FunctionUtils.closeFunction(joinFunction) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala index 17f8e1a233f10..66b60d44969fa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala @@ -22,8 +22,10 @@ import java.util import java.util.{Comparator, Optional} import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.state.{VoidNamespace, VoidNamespaceSerializer} import org.apache.flink.streaming.api.SimpleTimerService import org.apache.flink.streaming.api.operators._ @@ -115,12 +117,16 @@ class TemporalRowtimeJoin( private var joinFunction: FlatJoinFunction[Row, Row, Row] = _ override def open(): Unit = { + LOG.debug(s"Compiling FlatJoinFunction: $genJoinFuncName \n\n Code:\n$genJoinFuncCode") val clazz = compile( getRuntimeContext.getUserCodeClassLoader, genJoinFuncName, genJoinFuncCode) + LOG.debug("Instantiating FlatJoinFunction.") joinFunction = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(joinFunction, getRuntimeContext) + FunctionUtils.openFunction(joinFunction, new Configuration()) nextLeftIndex = getRuntimeContext.getState( new ValueStateDescriptor[JLong](NEXT_LEFT_INDEX_STATE_NAME, BasicTypeInfo.LONG_TYPE_INFO)) @@ -159,6 +165,22 @@ class TemporalRowtimeJoin( registerSmallestTimer(rowTime) // Timer to clean up the state } + override def onProcessingTime(timer: InternalTimer[Any, VoidNamespace]): Unit = { + throw new IllegalStateException("This should never happen") + } + + override def onEventTime(timer: InternalTimer[Any, VoidNamespace]): Unit = { + registeredTimer.clear() + val lastUnprocessedTime = emitResultAndCleanUpState(timerService.currentWatermark()) + if (lastUnprocessedTime < Long.MaxValue) { + registerTimer(lastUnprocessedTime) + } + } + + override def close(): Unit = { + FunctionUtils.closeFunction(joinFunction) + } + private def registerSmallestTimer(timestamp: Long): Unit = { val currentRegisteredTimer = registeredTimer.value() if (currentRegisteredTimer == null) { @@ -175,18 +197,6 @@ class TemporalRowtimeJoin( timerService.registerEventTimeTimer(timestamp) } - override def onProcessingTime(timer: InternalTimer[Any, VoidNamespace]): Unit = { - throw new IllegalStateException("This should never happen") - } - - override def onEventTime(timer: InternalTimer[Any, VoidNamespace]): Unit = { - registeredTimer.clear() - val lastUnprocessedTime = emitResultAndCleanUpState(timerService.currentWatermark()) - if (lastUnprocessedTime < Long.MaxValue) { - registerTimer(lastUnprocessedTime) - } - } - /** * @return a row time of the oldest unprocessed probe record or Long.MaxValue, if all records * have been processed. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala index a44bdef2e0907..ae7ca6a6032da 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala @@ -22,6 +22,7 @@ import java.util import java.util.{List => JList} import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.operators.join.JoinType @@ -110,6 +111,8 @@ abstract class TimeBoundedStreamJoin( genJoinFuncCode) LOG.debug("Instantiating JoinFunction.") joinFunction = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(joinFunction, getRuntimeContext) + FunctionUtils.openFunction(joinFunction, config) joinCollector = new EmitAwareCollector() joinCollector.setCRowChange(true) @@ -378,6 +381,10 @@ abstract class TimeBoundedStreamJoin( } } + override def close(): Unit = { + FunctionUtils.closeFunction(joinFunction) + } + /** * Calculate the expiration time with the given operator time and relative window size. * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/IterativeConditionRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/IterativeConditionRunner.scala index fc6755ea79fdd..15b9d37488948 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/IterativeConditionRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/IterativeConditionRunner.scala @@ -43,6 +43,7 @@ class IterativeConditionRunner( val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) LOG.debug("Instantiating IterativeCondition.") function = clazz.newInstance() + // TODO add logic for opening and closing the function once it can be a RichFunction } override def filter(value: Row, ctx: IterativeCondition.Context[Row]): Boolean = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/PatternSelectFunctionRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/PatternSelectFunctionRunner.scala index 643352271b5e3..c92cd38401215 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/PatternSelectFunctionRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/match/PatternSelectFunctionRunner.scala @@ -48,6 +48,7 @@ class PatternSelectFunctionRunner( val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) LOG.debug("Instantiating PatternSelectFunction.") function = clazz.newInstance() + // TODO add logic for opening and closing the function once it can be a RichFunction } override def flatSelect( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala index 9b0f08ea4a137..23ad51a0effa8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime import java.lang.Iterable +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{JoinFunction, RichGroupReduceFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.configuration.Configuration @@ -41,6 +42,12 @@ abstract class OuterJoinGroupReduceRunner( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating JoinFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, config) + } + + override def close(): Unit = { + FunctionUtils.closeFunction(function) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala index a9e0211c40fc8..fd3b46fbf632c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.runtime import java.lang.{Boolean => JBool} +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{JoinFunction, RichFlatJoinFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable @@ -45,9 +46,15 @@ abstract class OuterJoinRunner( val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating FlatJoinFunction.") function = clazz.newInstance() + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) } override def getProducedType: TypeInformation[Row] = returnType + + override def close(): Unit = { + FunctionUtils.closeFunction(function) + } } /** diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala index 6d02afca510a9..912bb047cba0b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala @@ -290,6 +290,9 @@ object Func19 extends ScalarFunction { } +/** + * A scalar function that always returns TRUE if opened correctly. + */ class Func20 extends ScalarFunction { private var permitted: Boolean = false diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/JoinITCase.scala index b7d349c29fcd4..2dbe1cddf00cd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/JoinITCase.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.Literal +import org.apache.flink.table.expressions.utils.Func20 import org.apache.flink.table.runtime.utils.TableProgramsClusterTestBase import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.table.utils.TableFunc2 @@ -53,7 +54,12 @@ class JoinITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) - val joinT = ds1.join(ds2).where('b === 'e).select('c, 'g) + val testOpenCall = new Func20 + + val joinT = ds1.join(ds2) + .where('b === 'e) + .where(testOpenCall('a + 'd)) + .select('c, 'g) val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" val results = joinT.toDataSet[Row].collect() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala index 197a039128818..7060d4db2a7c0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala @@ -27,6 +27,7 @@ import org.junit.Assert._ import org.junit.Test import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.expressions.utils.Func20 import org.apache.flink.table.expressions.{Literal, Null} import org.apache.flink.table.functions.aggfunctions.CountAggFunction import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg} @@ -225,7 +226,12 @@ class JoinITCase extends StreamingWithStateTestBase { val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h) - val joinT = ds1.join(ds2).where('b === 'e).select('c, 'g) + val testOpenCall = new Func20 + + val joinT = ds1.join(ds2) + .where('b === 'e) + .where(testOpenCall('a + 'd)) + .select('c, 'g) val expected = Seq("Hi,Hallo", "Hello,Hallo Welt", "Hello world,Hallo Welt") val results = joinT.toRetractStream[Row] From ca66bed090041e8a8c8bae2b7626a73ddf0ac6e0 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 6 Nov 2018 10:28:04 +0100 Subject: [PATCH 035/359] [FLINK-10765][test] Include s3p schema in S3 test This closes #7032. --- .../fs/s3presto/PrestoS3FileSystemITCase.java | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java index 07835740e2dc1..cc5c9935202a6 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java @@ -30,12 +30,16 @@ import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; import java.util.UUID; import static com.facebook.presto.hive.PrestoS3FileSystem.S3_USE_INSTANCE_CREDENTIALS; @@ -53,8 +57,17 @@ * consistency guarantees * and what the {@link com.facebook.presto.hive.PrestoS3FileSystem} offers. */ +@RunWith(Parameterized.class) public class PrestoS3FileSystemITCase extends TestLogger { + @Parameterized.Parameter + public String scheme; + + @Parameterized.Parameters(name = "Scheme = {0}") + public static List parameters() { + return Arrays.asList("s3", "s3p"); + } + private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); @@ -71,7 +84,7 @@ public static void checkIfCredentialsArePresent() { @Test public void testConfigKeysForwarding() throws Exception { - final Path path = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + final Path path = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR); // access without credentials should fail { @@ -145,7 +158,7 @@ public void testSimpleFileWriteAndRead() throws Exception { FileSystem.initialize(conf); - final Path path = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR + "/test.txt"); + final Path path = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR + "/test.txt"); final FileSystem fs = path.getFileSystem(); try { @@ -178,7 +191,7 @@ public void testDirectoryListing() throws Exception { FileSystem.initialize(conf); - final Path directory = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR + "/testdir/"); + final Path directory = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR + "/testdir/"); final FileSystem fs = directory.getFileSystem(); // directory must not yet exist From e34f8af3ef9502af6ece9d7c48e3bba249e55288 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 13:09:22 +0100 Subject: [PATCH 036/359] [FLINK-10361][tests][ES] Properly wait for ES to start This closes #7057. --- flink-end-to-end-tests/test-scripts/elasticsearch-common.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 16689c8a3f880..8bb9c42ab69cd 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -43,10 +43,10 @@ function wait_elasticsearch_working { echo "Waiting for Elasticsearch node to work..." for ((i=1;i<=60;i++)); do - curl -XGET 'http://localhost:9200' || true + output=$(curl -XGET 'http://localhost:9200' | grep "cluster_name" || true) # make sure the elasticsearch node is actually working - if [ $? -ne 0 ]; then + if [ "${output}" = "" ]; then sleep 1 else echo "Elasticsearch node is working." From 4a174f72cf7ed4e30cfba5594b72c35fb020ea8d Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 7 Nov 2018 15:22:36 +0100 Subject: [PATCH 037/359] [FLINK-10803][docs] Update the filesystem documentation for S3 (s3p). --- docs/ops/filesystems.md | 32 ++++++++++++++++++++++---------- 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/docs/ops/filesystems.md b/docs/ops/filesystems.md index 416302e49ca6e..77757b00dcf7f 100644 --- a/docs/ops/filesystems.md +++ b/docs/ops/filesystems.md @@ -22,32 +22,38 @@ specific language governing permissions and limitations under the License. --> -This page provides details on setting up and configuring distributed file systems for use with Flink. +This page provides details on setting up and configuring different file systems for use with Flink. +We start by describing how to use and configure the different file systems that are supported by Flink +out-of-the-box, before describing the necessary steps in order to add support about other/custom file system +implementations. ## Flink's File System support -Flink uses file systems both as a source and sink in streaming/batch applications, and as a target for checkpointing. +Flink uses file systems both as *sources* and *sinks* in streaming/batch applications and as a target for *checkpointing*. These file systems can for example be *Unix/Windows file systems*, *HDFS*, or even object stores like *S3*. The file system used for a specific file is determined by the file URI's scheme. For example `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` refers to a file in a specific HDFS cluster. File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify -files and objects in that file system. FileSystem instances are instantiates once per process and then cached / pooled, to -avoid configuration overhead per stream creation, and to enforce certain constraints, like connection/stream limits. +files and objects in that file system. FileSystem instances are instantiated once per process and then cached / pooled, to +avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits. ### Built-in File Systems -Flink directly implements the following file systems: +Flink ships with support for most of the popular file systems, namely *local*, *hadoop-compatible*, *S3*, *MapR FS* +and *OpenStack Swift FS*. Each of these is identified by the scheme included in the URI of the provide file path. + +Flink ships with implementations for the following file systems: - **local**: This file system is used when the scheme is *"file://"*, and it represents the file system of the local machine, including any NFS or SAN that is mounted into that local file system. - **S3**: Flink directly provides file systems to talk to Amazon S3. There are two alternative implementations, `flink-s3-fs-presto` - and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint, there is no need to add Hadoop to + and `flink-s3-fs-hadoop`. Both implementations are self-contained with no dependency footprint. There is no need to add Hadoop to the classpath to use them. Both internally use some Hadoop code, but "shade away" all classes to avoid any dependency conflicts. - - `flink-s3-fs-presto`, registered under the scheme *"s3://"*, is based on code from the [Presto project](https://prestodb.io/). + - `flink-s3-fs-presto`, registered under the scheme *"s3://"* and *"s3p://"*, is based on code from the [Presto project](https://prestodb.io/). You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.185/connector/hive.html#amazon-s3-configuration). - `flink-s3-fs-hadoop`, registered under *"s3://"* and *"s3a://"*, based on code from the [Hadoop Project](https://hadoop.apache.org/). @@ -56,7 +62,13 @@ including any NFS or SAN that is mounted into that local file system. To use those file systems when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-s3-fs-presto:{{ site.version }}` or `org.apache.flink:flink-s3-fs-hadoop:{{ site.version }}`). When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder. See also [AWS setup](deployment/aws.html) for additional details. - + + Attention: As described above, both Hadoop and Presto "listen" to paths with scheme set to *"s3://"*. This is + convenient for switching between implementations (Hadoop or Presto), but it can lead to non-determinism when both + implementations are required. This can happen when, for example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) + which only supports Hadoop, but uses Presto for checkpointing. In this case, it is advised to use explicitly *"s3a://"* + as a scheme for the sink (Hadoop) and *"s3p://"* for checkpointing (Presto). + - **MapR FS**: The MapR file system *"maprfs://"* is automatically available when the MapR libraries are in the classpath. - **OpenStack Swift FS**: Flink directly provides a file system to talk to the OpenStack Swift file system, registered under the scheme *"swift://"*. @@ -64,9 +76,9 @@ including any NFS or SAN that is mounted into that local file system. To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}` When starting a Flink application from the Flink binaries, copy or move the respective jar file from the `opt` folder to the `lib` folder. -### HDFS and Hadoop File System support +#### HDFS and Hadoop File System support -For all schemes where Flink cannot find a directly supported file system, Flink will try to use Hadoop to instantiate a file system for the respective scheme. +For all schemes where it cannot find a directly supported file system, Flink will try to use Hadoop to instantiate a file system for the respective scheme. All Hadoop file systems are automatically available once `flink-runtime` and the Hadoop libraries are in classpath. That way, Flink seamlessly supports all of Hadoop file systems, and all Hadoop-compatible file systems (HCFS), for example: From 87fe67955d268a0b34f5019938dacf4d69b67f54 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 7 Nov 2018 16:27:02 +0100 Subject: [PATCH 038/359] [FLINK-10803][docs] Update the StreamingFileSink documentation for S3. --- docs/dev/connectors/streamfile_sink.md | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index aea66c3cc48da..8f50675ccbc59 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -24,16 +24,25 @@ under the License. --> This connector provides a Sink that writes partitioned files to filesystems -supported by the Flink `FileSystem` abstraction. Since in streaming the input -is potentially infinite, the streaming file sink writes data into buckets. The -bucketing behaviour is configurable but a useful default is time-based +supported by the [Flink `FileSystem` abstraction]({{ site.baseurl}}/ops/filesystems.html). + +Important Note: For S3, the `StreamingFileSink` +supports only the [Hadoop-based](https://hadoop.apache.org/) FileSystem implementation, not +the implementation based on [Presto](https://prestodb.io/). In case your job uses the +`StreamingFileSink` to write to S3 but you want to use the Presto-based one for checkpointing, +it is advised to use explicitly *"s3a://"* (for Hadoop) as the scheme for the target path of +the sink and *"s3p://"* for checkpointing (for Presto). Using *"s3://"* for both the sink +and checkpointing may lead to unpredictable behavior, as both implementations "listen" to that scheme. + +Since in streaming the input is potentially infinite, the streaming file sink writes data +into buckets. The bucketing behaviour is configurable but a useful default is time-based bucketing where we start writing a new bucket every hour and thus get individual files that each contain a part of the infinite output stream. Within a bucket, we further split the output into smaller part files based on a rolling policy. This is useful to prevent individual bucket files from getting too big. This is also configurable but the default policy rolls files based on -file size and a timeout, i.e if no new data was written to a part file. +file size and a timeout, *i.e* if no new data was written to a part file. The `StreamingFileSink` supports both row-wise encoding formats and bulk-encoding formats, such as [Apache Parquet](http://parquet.apache.org). From bc2cb227090d4bdcf8eeb346d6737291a56ca551 Mon Sep 17 00:00:00 2001 From: vinoyang Date: Thu, 8 Nov 2018 21:37:39 +0800 Subject: [PATCH 039/359] [FLINK-10791] Provide end-to-end test for Kafka 0.11 connector (#7038) [FLINK-10791][e2e] Provide end-to-end test for Kafka 0.11 connector --- flink-dist/src/main/assemblies/bin.xml | 14 +++ .../run-pre-commit-tests.sh | 3 +- .../test-scripts/test_streaming_kafka011.sh | 24 +++++ .../pom.xml | 90 +++++++++++++++++++ .../examples/kafka/Kafka011Example.java | 69 ++++++++++++++ flink-examples/pom.xml | 1 + 6 files changed, 200 insertions(+), 1 deletion(-) create mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh create mode 100644 flink-examples/flink-examples-streaming-kafka-0.11/pom.xml create mode 100644 flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.java diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index e1adab537d968..fa1e80483415f 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -215,6 +215,20 @@ under the License. + + + ../flink-examples/flink-examples-streaming-kafka-0.11/target + examples/streaming + 0644 + + *.jar + + + flink-examples-streaming-kafka*.jar + original-*.jar + + + ../flink-libraries/flink-gelly-examples/target diff --git a/flink-end-to-end-tests/run-pre-commit-tests.sh b/flink-end-to-end-tests/run-pre-commit-tests.sh index 3e3494a491a49..c9fab30491645 100755 --- a/flink-end-to-end-tests/run-pre-commit-tests.sh +++ b/flink-end-to-end-tests/run-pre-commit-tests.sh @@ -53,7 +53,8 @@ run_test "State Evolution end-to-end test" "$END_TO_END_DIR/test-scripts/test_st run_test "Batch Python Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_python_wordcount.sh" run_test "Streaming Python Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_python_wordcount.sh" run_test "Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_wordcount.sh" -run_test "Kafka end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka010.sh" +run_test "Kafka 0.10 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka010.sh" +run_test "Kafka 0.11 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka011.sh" run_test "Modern Kafka end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka.sh" run_test "class loading end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_classloader.sh" run_test "Shaded Hadoop S3A end-to-end test" "$END_TO_END_DIR/test-scripts/test_shaded_hadoop_s3a.sh" diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh new file mode 100755 index 0000000000000..2e6e6d93debfc --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/kafka-common.sh 0.11.0.2 3.2.0 3.2 + +source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/Kafka011Example.jar + diff --git a/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml b/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml new file mode 100644 index 0000000000000..525e09489974c --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml @@ -0,0 +1,90 @@ + + + + + flink-examples + org.apache.flink + 1.7-SNAPSHOT + + 4.0.0 + + flink-examples-streaming-kafka-0.11 + flink-examples-streaming-kafka-0.11 + jar + + + + org.apache.flink + flink-examples-streaming-kafka-base + ${project.version} + + + + org.apache.flink + flink-connector-kafka-0.11_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + + org.apache.flink + flink-java + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + fat-jar-kafka-example + package + + shade + + + false + false + false + + + org.apache.flink.streaming.examples.kafka.Kafka011Example + + + Kafka011Example + + + + + + + + diff --git a/flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.java b/flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.java new file mode 100644 index 0000000000000..17b5d011f289b --- /dev/null +++ b/flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.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.streaming.examples.kafka; + +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.kafka.FlinkKafkaConsumer011; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011; +import org.apache.flink.streaming.examples.kafka.base.CustomWatermarkExtractor; +import org.apache.flink.streaming.examples.kafka.base.KafkaEvent; +import org.apache.flink.streaming.examples.kafka.base.KafkaEventSchema; +import org.apache.flink.streaming.examples.kafka.base.KafkaExampleUtil; +import org.apache.flink.streaming.examples.kafka.base.RollingAdditionMapper; + +/** + * A simple example that shows how to read from and write to Kafka. This will read String messages + * from the input topic, parse them into a POJO type {@link KafkaEvent}, group by some key, and finally + * perform a rolling addition on each key for which the results are written back to another topic. + * + *

This example also demonstrates using a watermark assigner to generate per-partition + * watermarks directly in the Flink Kafka consumer. For demonstration purposes, it is assumed that + * the String messages are of formatted as a (word,frequency,timestamp) tuple. + * + *

Example usage: + * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer + */ +public class Kafka011Example { + + public static void main(String[] args) throws Exception { + // parse input arguments + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + StreamExecutionEnvironment env = KafkaExampleUtil.prepareExecutionEnv(parameterTool); + + DataStream input = env + .addSource( + new FlinkKafkaConsumer011<>( + parameterTool.getRequired("input-topic"), + new KafkaEventSchema(), + parameterTool.getProperties()) + .assignTimestampsAndWatermarks(new CustomWatermarkExtractor())) + .keyBy("word") + .map(new RollingAdditionMapper()); + + input.addSink( + new FlinkKafkaProducer011<>( + parameterTool.getRequired("output-topic"), + new KafkaEventSchema(), + parameterTool.getProperties())); + + env.execute("Kafka 0.11 Example"); + } + +} diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 1c60ed63eccb4..9846128efa23f 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -39,6 +39,7 @@ under the License. flink-examples-streaming-kafka flink-examples-streaming-kafka-0.10 flink-examples-table + flink-examples-streaming-kafka-0.11 From 365e29d0715c9601e351bd76cc0fd0c327607e1b Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 8 Nov 2018 20:28:52 +0100 Subject: [PATCH 040/359] [hotfix][tests] Remove hard-coded scala versions --- .../flink-heavy-deployment-stress-test/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml index cc158a412283f..b3384defe1682 100644 --- a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml +++ b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml @@ -36,7 +36,7 @@ under the License. org.apache.flink - flink-streaming-java_2.11 + flink-streaming-java_${scala.binary.version} ${project.version} @@ -46,7 +46,7 @@ under the License. org.apache.flink - flink-test-utils_2.11 + flink-test-utils_${scala.binary.version} ${project.version} compile From d608eef43b5c79857f12e3203fd301e5d3006708 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 15:11:20 +0100 Subject: [PATCH 041/359] [hotfix][docs] Clarify unit for network.request-backoff --- docs/_includes/generated/task_manager_configuration.html | 4 ++-- .../org/apache/flink/configuration/TaskManagerOptions.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/_includes/generated/task_manager_configuration.html b/docs/_includes/generated/task_manager_configuration.html index 16c5d98f8be80..63bfb043ea44a 100644 --- a/docs/_includes/generated/task_manager_configuration.html +++ b/docs/_includes/generated/task_manager_configuration.html @@ -125,12 +125,12 @@

taskmanager.network.request-backoff.initial
100 - Minimum backoff for partition requests of input channels. + Minimum backoff in milliseconds for partition requests of input channels.
taskmanager.network.request-backoff.max
10000 - Maximum backoff for partition requests of input channels. + Maximum backoff in milliseconds for partition requests of input channels.
taskmanager.numberOfTaskSlots
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index 98fa47f669704..6d3ac4515b20f 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -316,7 +316,7 @@ public class TaskManagerOptions { key("taskmanager.network.request-backoff.initial") .defaultValue(100) .withDeprecatedKeys("taskmanager.net.request-backoff.initial") - .withDescription("Minimum backoff for partition requests of input channels."); + .withDescription("Minimum backoff in milliseconds for partition requests of input channels."); /** * Maximum backoff for partition requests of input channels. @@ -325,7 +325,7 @@ public class TaskManagerOptions { key("taskmanager.network.request-backoff.max") .defaultValue(10000) .withDeprecatedKeys("taskmanager.net.request-backoff.max") - .withDescription("Maximum backoff for partition requests of input channels."); + .withDescription("Maximum backoff in milliseconds for partition requests of input channels."); /** * Boolean flag to enable/disable more detailed metrics about inbound/outbound network queue From 8c3a93992618da0fdfe2c02abc5175609913c0cf Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 15:13:36 +0100 Subject: [PATCH 042/359] [FLINK-10825][tests] Increase request-backoff for high-parallelism e2e test --- .../test-scripts/test_high_parallelism_iterations.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh b/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh index 582b5a0af0d5f..93668d87a2f79 100755 --- a/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh +++ b/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh @@ -34,6 +34,7 @@ set_conf "taskmanager.memory.segment-size" "8kb" set_conf "taskmanager.network.netty.server.numThreads" "1" set_conf "taskmanager.network.netty.client.numThreads" "1" +set_conf "taskmanager.network.request-backoff.max" "60000" set_conf "taskmanager.numberOfTaskSlots" "1" From b1e2406a65790bae3764f49918599d40dfca5691 Mon Sep 17 00:00:00 2001 From: Hequn Cheng Date: Fri, 9 Nov 2018 14:29:24 +0800 Subject: [PATCH 043/359] [FLINK-10814][examples] Add scala suffix to Kafka example module --- .../pom.xml | 49 ++++++++++++++++++- .../examples/kafka/Kafka010Example.scala | 2 +- .../pom.xml | 4 +- .../pom.xml | 2 +- .../flink-examples-streaming-kafka/pom.xml | 4 +- 5 files changed, 53 insertions(+), 8 deletions(-) diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml b/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml index d9550f6ef2f9f..fa09fc22dec75 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml @@ -27,12 +27,13 @@ under the License. 4.0.0 - flink-examples-streaming-kafka-0.10 + flink-examples-streaming-kafka-0.10_${scala.binary.version} + flink-examples-streaming-kafka-0.10 org.apache.flink - flink-examples-streaming-kafka-base + flink-examples-streaming-kafka-base_${scala.binary.version} ${project.version} @@ -48,6 +49,12 @@ under the License. ${project.version} + + org.apache.flink + flink-streaming-scala_${scala.binary.version} + ${project.version} + + org.apache.flink flink-java @@ -57,6 +64,44 @@ under the License. + + + + net.alchim31.maven + scala-maven-plugin + + + + scala-compile-first + process-resources + + add-source + compile + + + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + + + org.scalastyle + scalastyle-maven-plugin + + ${project.basedir}/../../tools/maven/scalastyle-config.xml + + + org.apache.maven.plugins diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala b/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala index c2ea5617a685f..9f4fdc4c29489 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala +++ b/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala @@ -21,7 +21,7 @@ package org.apache.flink.streaming.scala.examples.kafka import org.apache.flink.api.common.restartstrategy.RestartStrategies import org.apache.flink.api.common.serialization.SimpleStringSchema import org.apache.flink.api.java.utils.ParameterTool -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.scala._ import org.apache.flink.streaming.connectors.kafka.{FlinkKafkaConsumer010, FlinkKafkaProducer010} /** diff --git a/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml b/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml index 525e09489974c..c044767a22186 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml @@ -27,14 +27,14 @@ under the License. 4.0.0 - flink-examples-streaming-kafka-0.11 + flink-examples-streaming-kafka-0.11_${scala.binary.version} flink-examples-streaming-kafka-0.11 jar org.apache.flink - flink-examples-streaming-kafka-base + flink-examples-streaming-kafka-base_${scala.binary.version} ${project.version} diff --git a/flink-examples/flink-examples-streaming-kafka-base/pom.xml b/flink-examples/flink-examples-streaming-kafka-base/pom.xml index 25f533c9ad279..3238d68f35490 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka-base/pom.xml @@ -27,7 +27,7 @@ under the License. 4.0.0 - flink-examples-streaming-kafka-base + flink-examples-streaming-kafka-base_${scala.binary.version} flink-examples-streaming-kafka-base jar diff --git a/flink-examples/flink-examples-streaming-kafka/pom.xml b/flink-examples/flink-examples-streaming-kafka/pom.xml index fe58baa7ef30a..b02eaa588684a 100644 --- a/flink-examples/flink-examples-streaming-kafka/pom.xml +++ b/flink-examples/flink-examples-streaming-kafka/pom.xml @@ -27,13 +27,13 @@ under the License. 4.0.0 - flink-examples-streaming-kafka + flink-examples-streaming-kafka_${scala.binary.version} flink-examples-streaming-kafka org.apache.flink - flink-examples-streaming-kafka-base + flink-examples-streaming-kafka-base_${scala.binary.version} ${project.version} From 365eece2b70b99b0a68edb78b6c9abc2ea84cf11 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 11:30:16 +0100 Subject: [PATCH 044/359] [FLINK-10823][jdbc] Add scala suffix --- flink-connectors/flink-jdbc/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-jdbc/pom.xml b/flink-connectors/flink-jdbc/pom.xml index 96c984f65beb8..9f34c4935dc25 100644 --- a/flink-connectors/flink-jdbc/pom.xml +++ b/flink-connectors/flink-jdbc/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-jdbc + flink-jdbc_${scala.binary.version} flink-jdbc jar From 813212926f266a5095e51225cef35347daa85b65 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 11:30:36 +0100 Subject: [PATCH 045/359] [FLINK-10823][metrics][prometheus] Add scala suffix --- docs/monitoring/metrics.md | 2 +- flink-dist/pom.xml | 2 +- flink-dist/src/main/assemblies/opt.xml | 2 +- flink-docs/pom.xml | 2 +- .../flink-metrics-reporter-prometheus-test/pom.xml | 2 +- flink-metrics/flink-metrics-prometheus/pom.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 04efdef26fc2a..e482df2e79579 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -674,7 +674,7 @@ metrics.reporter.grph.protocol: TCP ### Prometheus (org.apache.flink.metrics.prometheus.PrometheusReporter) -In order to use this reporter you must copy `/opt/flink-metrics-prometheus-{{site.version}}.jar` into the `/lib` folder +In order to use this reporter you must copy `/opt/flink-metrics-prometheus{{site.scala_version_suffix}}-{{site.version}}.jar` into the `/lib` folder of your Flink distribution. Parameters: diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index c11517af53f93..92488c838b7c0 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -244,7 +244,7 @@ under the License. org.apache.flink - flink-metrics-prometheus + flink-metrics-prometheus_${scala.binary.version} ${project.version} provided diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml index 6fe661a6c46da..8d153766bb690 100644 --- a/flink-dist/src/main/assemblies/opt.xml +++ b/flink-dist/src/main/assemblies/opt.xml @@ -106,7 +106,7 @@ - ../flink-metrics/flink-metrics-prometheus/target/flink-metrics-prometheus-${project.version}.jar + ../flink-metrics/flink-metrics-prometheus/target/flink-metrics-prometheus_${scala.binary.version}-${project.version}.jar opt/ flink-metrics-prometheus-${project.version}.jar 0644 diff --git a/flink-docs/pom.xml b/flink-docs/pom.xml index 75e3f9b53fb98..5c03fff2301bc 100644 --- a/flink-docs/pom.xml +++ b/flink-docs/pom.xml @@ -55,7 +55,7 @@ under the License. org.apache.flink - flink-metrics-prometheus + flink-metrics-prometheus_${scala.binary.version} ${project.version} diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml index ddeac14d6de15..844c9b09346a8 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml @@ -39,7 +39,7 @@ under the License. org.apache.flink - flink-metrics-prometheus + flink-metrics-prometheus_${scala.binary.version} ${project.version} test diff --git a/flink-metrics/flink-metrics-prometheus/pom.xml b/flink-metrics/flink-metrics-prometheus/pom.xml index 145113d9999eb..430753e8d1e5e 100644 --- a/flink-metrics/flink-metrics-prometheus/pom.xml +++ b/flink-metrics/flink-metrics-prometheus/pom.xml @@ -29,7 +29,7 @@ under the License. .. - flink-metrics-prometheus + flink-metrics-prometheus_${scala.binary.version} flink-metrics-prometheus From b95d69ae4199dcbb0886a96905bdfec38b9988a5 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 11:32:45 +0100 Subject: [PATCH 046/359] [FLINK-10823][metrics][jmx] Add scala suffix --- flink-connectors/flink-connector-kafka-0.10/pom.xml | 2 +- flink-connectors/flink-connector-kafka-0.11/pom.xml | 2 +- flink-connectors/flink-connector-kafka-0.8/pom.xml | 2 +- flink-connectors/flink-connector-kafka-0.9/pom.xml | 2 +- flink-connectors/flink-connector-kafka-base/pom.xml | 2 +- flink-connectors/flink-connector-kafka/pom.xml | 2 +- flink-dist/pom.xml | 2 +- flink-metrics/flink-metrics-jmx/pom.xml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 135dc59b65543..8e2f338fbbde4 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -187,7 +187,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml index bf04aebb390fb..d8ae959376489 100644 --- a/flink-connectors/flink-connector-kafka-0.11/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml @@ -196,7 +196,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml index 52f0ef17b9ad9..50f41d0815a69 100644 --- a/flink-connectors/flink-connector-kafka-0.8/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml @@ -176,7 +176,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index 9b9bbefa7b3f7..d68a2e68d60f4 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -137,7 +137,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml index a0feec90b55a3..4921550c0996b 100644 --- a/flink-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-connectors/flink-connector-kafka-base/pom.xml @@ -153,7 +153,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index 10f738337201d..91b8706a035c8 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -205,7 +205,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} test diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 92488c838b7c0..d58f6f801479f 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -106,7 +106,7 @@ under the License. org.apache.flink - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} ${project.version} diff --git a/flink-metrics/flink-metrics-jmx/pom.xml b/flink-metrics/flink-metrics-jmx/pom.xml index 750ab620a2f6b..ccf90ef07375a 100644 --- a/flink-metrics/flink-metrics-jmx/pom.xml +++ b/flink-metrics/flink-metrics-jmx/pom.xml @@ -29,7 +29,7 @@ under the License. .. - flink-metrics-jmx + flink-metrics-jmx_${scala.binary.version} flink-metrics-jmx From 6d138a84e3feab488d679716e1475f95508d2662 Mon Sep 17 00:00:00 2001 From: vinoyang Date: Fri, 9 Nov 2018 16:34:32 +0800 Subject: [PATCH 047/359] [FLINK-10733][tests] Inline clean_log_files() --- flink-end-to-end-tests/test-scripts/common.sh | 5 ----- flink-end-to-end-tests/test-scripts/test-runner-common.sh | 3 ++- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 36cc0df142ce1..5d45554059852 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -645,11 +645,6 @@ function clean_stdout_files { echo "Deleted all stdout files under ${FLINK_DIR}/log/" } -function clean_log_files { - rm ${FLINK_DIR}/log/* - echo "Deleted all files under ${FLINK_DIR}/log/" -} - # Expect a string to appear in the log files of the task manager before a given timeout # $1: expected string # $2: timeout in seconds diff --git a/flink-end-to-end-tests/test-scripts/test-runner-common.sh b/flink-end-to-end-tests/test-scripts/test-runner-common.sh index 44ec80964bbbc..1eef75c5acd85 100644 --- a/flink-end-to-end-tests/test-scripts/test-runner-common.sh +++ b/flink-end-to-end-tests/test-scripts/test-runner-common.sh @@ -101,7 +101,8 @@ function cleanup_proc { # Cleans up all temporary folders and files function cleanup_tmp_files { - clean_log_files + rm ${FLINK_DIR}/log/* + echo "Deleted all files under ${FLINK_DIR}/log/" rm -rf ${TEST_DATA_DIR} 2> /dev/null echo "Deleted ${TEST_DATA_DIR}" From 789d8a79612d13d579674a18b552abab414f23a9 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 8 Nov 2018 11:13:29 +0100 Subject: [PATCH 048/359] [FLINK-10813][travis][scala] Automatically check scala-suffixes --- tools/travis_controller.sh | 13 +++++++++++++ tools/verify_scala_suffixes.sh | 11 +++++++++-- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/tools/travis_controller.sh b/tools/travis_controller.sh index 7b2c8fd6a011e..6aa08e0d659b8 100755 --- a/tools/travis_controller.sh +++ b/tools/travis_controller.sh @@ -94,6 +94,19 @@ if [ $STAGE == "$STAGE_COMPILE" ]; then $MVN EXIT_CODE=$? + if [ $EXIT_CODE == 0 ]; then + printf "\n\n==============================================================================\n" + printf "Checking scala suffixes\n" + printf "==============================================================================\n" + + ./tools/verify_scala_suffixes.sh + EXIT_CODE=$? + else + printf "\n==============================================================================\n" + printf "Previous build failure detected, skipping scala-suffixes check.\n" + printf "==============================================================================\n" + fi + if [ $EXIT_CODE == 0 ]; then printf "\n\n==============================================================================\n" printf "Checking dependency convergence\n" diff --git a/tools/verify_scala_suffixes.sh b/tools/verify_scala_suffixes.sh index 4d7b90c44be25..e4099de5fc7b5 100755 --- a/tools/verify_scala_suffixes.sh +++ b/tools/verify_scala_suffixes.sh @@ -79,6 +79,13 @@ in_block=0 block_name="" block_infected=0 +# exclude e2e modules and flink-docs for convenience as they +# a) are not deployed during a release +# b) exist only for dev purposes +# c) no-one should depend on them +e2e_modules=$(find flink-end-to-end-tests -mindeptha 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',') +excluded_modules=\!${e2e_modules//,/,\!},!flink-docs + echo "Analyzing modules for Scala dependencies using 'mvn dependency:tree'." echo "If you haven't built the project, please do so first by running \"mvn clean install -DskipTests\"" @@ -102,14 +109,14 @@ while read line; do block_name="" block_infected=0 elif [[ $in_block -eq 1 ]]; then - echo $line | grep org.scala-lang >/dev/null + echo $line | grep "org.scala-lang" | grep --invert-match "org.scala-lang.*:.*:.*:test" >/dev/null if [ $? -eq 0 ]; then #echo $block_name infected="$block_name $infected" block_infected=1 fi fi -done < <(mvn -o dependency:tree -Dincludes=org.scala-lang | tee /dev/tty) +done < <(mvn -o dependency:tree -Dincludes=org.scala-lang -pl ${excluded_modules} | tee /dev/tty) # deduplicate and sort From 158e2c89e15eeed1dad1404b66709c5854d2049f Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 9 Nov 2018 09:52:15 +0100 Subject: [PATCH 049/359] [FLINK-10801][e2e] Retry verify_result_hash in elastichsearch-common (#7060) Instead of looping the verification until the expected number of results loop until we get the correct output. This tries to solve the problem of some records (aggregated? updated?) arriving later. --- flink-end-to-end-tests/test-scripts/common.sh | 15 +++++++-- .../test-scripts/elasticsearch-common.sh | 31 +++++++++++++------ 2 files changed, 34 insertions(+), 12 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 5d45554059852..41dfe0fc4cbbe 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -451,6 +451,16 @@ function cancel_job { } function check_result_hash { + local error_code=0 + check_result_hash_no_exit "$@" || error_code=$? + + if [ "$error_code" != "0" ] + then + exit $error_code + fi +} + +function check_result_hash_no_exit { local name=$1 local outfile_prefix=$2 local expected=$3 @@ -462,18 +472,19 @@ function check_result_hash { actual=$(LC_ALL=C sort $outfile_prefix* | md5sum | awk '{print $1}') else echo "Neither 'md5' nor 'md5sum' binary available." - exit 2 + return 2 fi if [[ "$actual" != "$expected" ]] then echo "FAIL $name: Output hash mismatch. Got $actual, expected $expected." echo "head hexdump of actual:" head $outfile_prefix* | hexdump -c - exit 1 + return 1 else echo "pass $name" # Output files are left behind in /tmp fi + return 0 } # This function starts the given number of task managers and monitors their processes. diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 8bb9c42ab69cd..695905222f553 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -83,23 +83,34 @@ function verify_result_hash { local name=$1 local index=$2 local numRecords=$3 - local hash=$4 + local expectedHash=$4 - while : ; do + for i in {1..30} + do + local error_code=0 + + echo "Result verification attempt $i..." curl "localhost:9200/${index}/_search?q=*&pretty" > $TEST_DATA_DIR/es_output || true - if [ -n "$(grep "\"total\" : $numRecords" $TEST_DATA_DIR/es_output)" ]; then - break - else - echo "Waiting for Elasticsearch records ..." + # remove meta information + sed '2,9d' $TEST_DATA_DIR/es_output > $TEST_DATA_DIR/es_content + + check_result_hash_no_exit "$name" $TEST_DATA_DIR/es_content "$expectedHash" || error_code=$? + + if [ "$error_code" != "0" ] + then + echo "Result verification attempt $i has failed" sleep 1 + else + break fi done - # remove meta information - sed '2,9d' $TEST_DATA_DIR/es_output > $TEST_DATA_DIR/es_content - - check_result_hash "$name" $TEST_DATA_DIR/es_content "$hash" + if [ "$error_code" != "0" ] + then + echo "All result verification attempts have failed" + exit $error_code + fi } function shutdown_elasticsearch_cluster { From 822023cba2a3d7ab4fb3699fef924a8f3253a574 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 6 Nov 2018 12:54:13 +0800 Subject: [PATCH 050/359] [FLINK-10789] [cep] LockableTypeSerializerSnapshot should be a TypeSerializerSnapshot This also adds a migration test for the LockableTypeSerializer. --- flink-libraries/flink-cep/pom.xml | 8 ++ .../flink/cep/nfa/sharedbuffer/Lockable.java | 50 ++++------ .../LockableSerializerConfigSnapshot.java | 46 --------- .../LockableTypeSerializerSnapshot.java | 92 ++++++++++++++++++ ...leTypeSerializerSnapshotMigrationTest.java | 43 ++++++++ .../flink-1.6-lockable-type-serializer-data | Bin 0 -> 160 bytes ...link-1.6-lockable-type-serializer-snapshot | Bin 0 -> 561 bytes 7 files changed, 162 insertions(+), 77 deletions(-) delete mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshot.java create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshotMigrationTest.java create mode 100644 flink-libraries/flink-cep/src/test/resources/flink-1.6-lockable-type-serializer-data create mode 100644 flink-libraries/flink-cep/src/test/resources/flink-1.6-lockable-type-serializer-snapshot diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml index 81db582b64060..7c32735d92c67 100644 --- a/flink-libraries/flink-cep/pom.xml +++ b/flink-libraries/flink-cep/pom.xml @@ -59,6 +59,14 @@ under the License. + + org.apache.flink + flink-core + ${project.version} + test-jar + test + + org.apache.flink flink-test-utils_${scala.binary.version} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java index 4ba4592fd9927..f7bc968ce0922 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java @@ -26,7 +26,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -187,40 +186,29 @@ public boolean canEqual(Object obj) { } @Override - public TypeSerializerConfigSnapshot> snapshotConfiguration() { - return new LockableSerializerConfigSnapshot<>(elementSerializer); + public TypeSerializerSnapshot> snapshotConfiguration() { + return new LockableTypeSerializerSnapshot<>(elementSerializer); } + /** + * This cannot be removed until {@link TypeSerializerConfigSnapshot} is no longer supported. + */ @Override public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - if (configSnapshot instanceof LockableSerializerConfigSnapshot) { - @SuppressWarnings("unchecked") - LockableSerializerConfigSnapshot snapshot = (LockableSerializerConfigSnapshot) configSnapshot; - - Tuple2, TypeSerializerSnapshot> nestedSerializerAndConfig = - snapshot.getSingleNestedSerializerAndConfig(); - - CompatibilityResult inputCompatibilityResult = CompatibilityUtil.resolveCompatibilityResult( - nestedSerializerAndConfig.f0, - UnloadableDummyTypeSerializer.class, - nestedSerializerAndConfig.f1, - elementSerializer); - - return (inputCompatibilityResult.isRequiresMigration()) - ? CompatibilityResult.requiresMigration() - : CompatibilityResult.compatible(); - } else { - // backwards compatibility path - CompatibilityResult inputCompatibilityResult = CompatibilityUtil.resolveCompatibilityResult( - configSnapshot.restoreSerializer(), - UnloadableDummyTypeSerializer.class, - configSnapshot, - elementSerializer); - - return (inputCompatibilityResult.isRequiresMigration()) - ? CompatibilityResult.requiresMigration() - : CompatibilityResult.compatible(); - } + // backwards compatibility path + CompatibilityResult inputCompatibilityResult = CompatibilityUtil.resolveCompatibilityResult( + configSnapshot.restoreSerializer(), + UnloadableDummyTypeSerializer.class, + configSnapshot, + elementSerializer); + + return (inputCompatibilityResult.isRequiresMigration()) + ? CompatibilityResult.requiresMigration() + : CompatibilityResult.compatible(); + } + + TypeSerializer getElementSerializer() { + return elementSerializer; } @VisibleForTesting diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java deleted file mode 100644 index 9e78bc013ef2a..0000000000000 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java +++ /dev/null @@ -1,46 +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.cep.nfa.sharedbuffer; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; - -/** - * A {@link TypeSerializerConfigSnapshot} for the {@link Lockable.LockableTypeSerializer}. - */ -@Internal -public class LockableSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { - - private static final int VERSION = 1; - - /** This empty nullary constructor is required for deserializing the configuration. */ - public LockableSerializerConfigSnapshot() {} - - public LockableSerializerConfigSnapshot(TypeSerializer elementSerializer) { - super(elementSerializer); - } - - @Override - public int getVersion() { - return VERSION; - } - -} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshot.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshot.java new file mode 100644 index 0000000000000..44a4670cc1da8 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshot.java @@ -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.cep.nfa.sharedbuffer; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A {@link TypeSerializerSnapshot} for the {@link Lockable.LockableTypeSerializer}. + */ +@Internal +public class LockableTypeSerializerSnapshot implements TypeSerializerSnapshot> { + + private static final int CURRENT_VERSION = 1; + + private CompositeSerializerSnapshot nestedElementSerializerSnapshot; + + /** + * Constructor for read instantiation. + */ + public LockableTypeSerializerSnapshot() {} + + /** + * Constructor to create the snapshot for writing. + */ + public LockableTypeSerializerSnapshot(TypeSerializer elementSerializer) { + this.nestedElementSerializerSnapshot = new CompositeSerializerSnapshot(Preconditions.checkNotNull(elementSerializer)); + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public TypeSerializer> restoreSerializer() { + return new Lockable.LockableTypeSerializer<>(nestedElementSerializerSnapshot.getRestoreSerializer(0)); + } + + @Override + public TypeSerializerSchemaCompatibility> resolveSchemaCompatibility(TypeSerializer> newSerializer) { + checkState(nestedElementSerializerSnapshot != null); + + if (newSerializer instanceof Lockable.LockableTypeSerializer) { + Lockable.LockableTypeSerializer serializer = (Lockable.LockableTypeSerializer) newSerializer; + + return nestedElementSerializerSnapshot.resolveCompatibilityWithNested( + TypeSerializerSchemaCompatibility.compatibleAsIs(), + serializer.getElementSerializer()); + } + else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + nestedElementSerializerSnapshot.writeCompositeSnapshot(out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.nestedElementSerializerSnapshot = CompositeSerializerSnapshot.readCompositeSnapshot(in, userCodeClassLoader); + } + +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshotMigrationTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshotMigrationTest.java new file mode 100644 index 0000000000000..bb3b7f27e108b --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/sharedbuffer/LockableTypeSerializerSnapshotMigrationTest.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.cep.nfa.sharedbuffer; + +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; +import org.apache.flink.api.common.typeutils.base.StringSerializer; + +/** + * Migration test for the {@link LockableTypeSerializerSnapshot}. + */ +public class LockableTypeSerializerSnapshotMigrationTest extends TypeSerializerSnapshotMigrationTestBase> { + + private static final String DATA = "flink-1.6-lockable-type-serializer-data"; + private static final String SNAPSHOT = "flink-1.6-lockable-type-serializer-snapshot"; + + public LockableTypeSerializerSnapshotMigrationTest() { + super( + TestSpecification.>builder( + "1.6-lockable-type-serializer", + Lockable.LockableTypeSerializer.class, + LockableTypeSerializerSnapshot.class) + .withSerializerProvider(() -> new Lockable.LockableTypeSerializer<>(StringSerializer.INSTANCE)) + .withSnapshotDataLocation(SNAPSHOT) + .withTestData(DATA, 10) + ); + } +} diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-lockable-type-serializer-data b/flink-libraries/flink-cep/src/test/resources/flink-1.6-lockable-type-serializer-data new file mode 100644 index 0000000000000000000000000000000000000000..b17504031a6459f73bab729ec67ccb654c60b84b GIT binary patch literal 160 bcmZQzU|`_MNX^N~S18Xf%1Hro2NfRxZ-Xom literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-cep/src/test/resources/flink-1.6-lockable-type-serializer-snapshot b/flink-libraries/flink-cep/src/test/resources/flink-1.6-lockable-type-serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..3d9fa0c07fbc36255e05e4803f96b05c27c1d6e8 GIT binary patch literal 561 zcmbu6F-`+95JhKK;s6vJ04Xiw2GOJgMKoI=Ef?^b@y3eRUdE0{5Td1^aBoO!2B-NAP0`vrad B!CL?T literal 0 HcmV?d00001 From 5ce0a912b627a3f23ed38f87129bbfa7e8948c33 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 6 Nov 2018 13:48:37 +0800 Subject: [PATCH 051/359] [FLINK-10789] [table] ListViewSerializerSnapshot should be a TypeSerializerSnapshot This also adds a migration test for the ListViewSerializer. --- .../dataview/ListViewSerializerSnapshot.java | 93 ++++++++++++++++++ .../table/dataview/ListViewSerializer.scala | 34 +++---- ...tViewSerializerSnapshotMigrationTest.java} | 33 +++---- .../flink-1.6-list-view-serializer-data | Bin 0 -> 240 bytes .../flink-1.6-list-view-serializer-snapshot | Bin 0 -> 1038 bytes 5 files changed, 121 insertions(+), 39 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerSnapshot.java rename flink-libraries/flink-table/src/{main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java => test/java/org/apache/flink/table/dataview/ListViewSerializerSnapshotMigrationTest.java} (52%) create mode 100644 flink-libraries/flink-table/src/test/resources/flink-1.6-list-view-serializer-data create mode 100644 flink-libraries/flink-table/src/test/resources/flink-1.6-list-view-serializer-snapshot diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerSnapshot.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerSnapshot.java new file mode 100644 index 0000000000000..cca84d28d17db --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerSnapshot.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.table.dataview; + +import org.apache.flink.api.common.typeutils.CompositeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.api.dataview.ListView; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A {@link TypeSerializerSnapshot} for the {@link ListViewSerializer}. + * + * @param the type of the list elements. + */ +public final class ListViewSerializerSnapshot implements TypeSerializerSnapshot> { + + private static final int CURRENT_VERSION = 1; + + private CompositeSerializerSnapshot nestedListSerializerSnapshot; + + /** + * Constructor for read instantiation. + */ + public ListViewSerializerSnapshot() {} + + /** + * Constructor to create the snapshot for writing. + */ + public ListViewSerializerSnapshot(TypeSerializer> listSerializer) { + this.nestedListSerializerSnapshot = new CompositeSerializerSnapshot(Preconditions.checkNotNull(listSerializer)); + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public TypeSerializer> restoreSerializer() { + return new ListViewSerializer<>(nestedListSerializerSnapshot.getRestoreSerializer(0)); + } + + @Override + public TypeSerializerSchemaCompatibility> resolveSchemaCompatibility(TypeSerializer> newSerializer) { + checkState(nestedListSerializerSnapshot != null); + + if (newSerializer instanceof ListViewSerializer) { + ListViewSerializer serializer = (ListViewSerializer) newSerializer; + + return nestedListSerializerSnapshot.resolveCompatibilityWithNested( + TypeSerializerSchemaCompatibility.compatibleAsIs(), + serializer.getListSerializer()); + } + else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + nestedListSerializerSnapshot.writeCompositeSnapshot(out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.nestedListSerializerSnapshot = CompositeSerializerSnapshot.readCompositeSnapshot(in, userCodeClassLoader); + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala index 9a10c118c0b65..246af6c0dab33 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala @@ -33,13 +33,14 @@ import org.apache.flink.table.api.dataview.ListView * @param listSerializer List serializer. * @tparam T The type of element in the list. */ -class ListViewSerializer[T](val listSerializer: ListSerializer[T]) +@SerialVersionUID(-2030398712359267867L) +class ListViewSerializer[T](val listSerializer: TypeSerializer[java.util.List[T]]) extends TypeSerializer[ListView[T]] { override def isImmutableType: Boolean = false override def duplicate(): TypeSerializer[ListView[T]] = { - new ListViewSerializer[T](listSerializer.duplicate().asInstanceOf[ListSerializer[T]]) + new ListViewSerializer[T](listSerializer.duplicate()) } override def createInstance(): ListView[T] = { @@ -75,41 +76,28 @@ class ListViewSerializer[T](val listSerializer: ListSerializer[T]) override def equals(obj: Any): Boolean = canEqual(this) && listSerializer.equals(obj.asInstanceOf[ListViewSerializer[_]].listSerializer) - override def snapshotConfiguration(): ListViewSerializerConfigSnapshot[T] = - new ListViewSerializerConfigSnapshot[T](listSerializer) + override def snapshotConfiguration(): ListViewSerializerSnapshot[T] = + new ListViewSerializerSnapshot[T](listSerializer) override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot[_]): CompatibilityResult[ListView[T]] = { configSnapshot match { - case snapshot: ListViewSerializerConfigSnapshot[T] => - val previousListSerializerAndConfig = - snapshot.getSingleNestedSerializerAndConfig - - val compatResult = CompatibilityUtil.resolveCompatibilityResult( - previousListSerializerAndConfig.f0, - classOf[UnloadableDummyTypeSerializer[_]], - previousListSerializerAndConfig.f1, - listSerializer) - - if (!compatResult.isRequiresMigration) { - CompatibilityResult.compatible[ListView[T]] - } else { - CompatibilityResult.requiresMigration[ListView[T]] - } - // backwards compatibility path; - // Flink versions older or equal to 1.5.x returns a + // Flink versions older or equal to 1.6.x returns a // CollectionSerializerConfigSnapshot as the snapshot case legacySnapshot: CollectionSerializerConfigSnapshot[java.util.List[T], T] => val previousListSerializerAndConfig = legacySnapshot.getSingleNestedSerializerAndConfig + // in older versions, the nested list serializer was always + // specifically a ListSerializer, so this cast is safe + val castedSer = listSerializer.asInstanceOf[ListSerializer[T]] val compatResult = CompatibilityUtil.resolveCompatibilityResult( previousListSerializerAndConfig.f0, classOf[UnloadableDummyTypeSerializer[_]], previousListSerializerAndConfig.f1, - listSerializer.getElementSerializer) + castedSer.getElementSerializer) if (!compatResult.isRequiresMigration) { CompatibilityResult.compatible[ListView[T]] @@ -120,4 +108,6 @@ class ListViewSerializer[T](val listSerializer: ListSerializer[T]) case _ => CompatibilityResult.requiresMigration[ListView[T]] } } + + def getListSerializer: TypeSerializer[java.util.List[T]] = listSerializer } diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/ListViewSerializerSnapshotMigrationTest.java similarity index 52% rename from flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java rename to flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/ListViewSerializerSnapshotMigrationTest.java index b2126028de2a0..5465ada23ff28 100644 --- a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/ListViewSerializerSnapshotMigrationTest.java @@ -18,29 +18,28 @@ package org.apache.flink.table.dataview; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.table.api.dataview.ListView; /** - * A {@link TypeSerializerConfigSnapshot} for the {@link ListViewSerializer}. - * - * @param the type of the list elements. + * Migration test for the {@link ListViewSerializerSnapshot}. */ -public final class ListViewSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { - - private static final int VERSION = 1; +public class ListViewSerializerSnapshotMigrationTest extends TypeSerializerSnapshotMigrationTestBase> { - /** This empty nullary constructor is required for deserializing the configuration. */ - public ListViewSerializerConfigSnapshot() {} - - public ListViewSerializerConfigSnapshot(ListSerializer listSerializer) { - super(listSerializer); - } + private static final String DATA = "flink-1.6-list-view-serializer-data"; + private static final String SNAPSHOT = "flink-1.6-list-view-serializer-snapshot"; - @Override - public int getVersion() { - return VERSION; + public ListViewSerializerSnapshotMigrationTest() { + super( + TestSpecification.>builder( + "1.6-list-view-serializer", + ListViewSerializer.class, + ListViewSerializerSnapshot.class) + .withSerializerProvider(() -> new ListViewSerializer<>(new ListSerializer<>(StringSerializer.INSTANCE))) + .withSnapshotDataLocation(SNAPSHOT) + .withTestData(DATA, 10) + ); } } diff --git a/flink-libraries/flink-table/src/test/resources/flink-1.6-list-view-serializer-data b/flink-libraries/flink-table/src/test/resources/flink-1.6-list-view-serializer-data new file mode 100644 index 0000000000000000000000000000000000000000..7b6c68ac7f38ed5bd67637f348e447945807c1fc GIT binary patch literal 240 icmZQzU|?ZM%g<*?N-Sc_NX^N~XDiPy%1L1urs4oe3q%6| literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-table/src/test/resources/flink-1.6-list-view-serializer-snapshot b/flink-libraries/flink-table/src/test/resources/flink-1.6-list-view-serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..2e2fd8ab9f10abdc01fba3081d1fa6b2bad8383d GIT binary patch literal 1038 zcmd6ly-EZz6ov1M*eVucCHMpx5D{cL7qJ;Z8L`i~B!zWX-=abk&4MT{KL9$3_n zc`ij{x&@vZ#&uMu8lw{KhvSZdwI6!yG^wV^_cYnJ96wn3gCGRx6xP(nWROyV^XEzP zSLaX94@-B>+X8aTQH&@wus)%^ae;B?1TG-7(&8zkrA9H4d2ypQ6>LsH8dsBADn}zl z_vZ5Up0u~h+?+sHj-4sMHOV Date: Tue, 6 Nov 2018 14:04:27 +0800 Subject: [PATCH 052/359] [FLINK-10789] [table] MapViewSerializerSnapshot should be a TypeSerializerSnapshot This also adds a migration test for the MapViewSerializer. --- .../MapViewSerializerConfigSnapshot.java | 47 --------- .../dataview/MapViewSerializerSnapshot.java | 95 ++++++++++++++++++ .../table/dataview/MapViewSerializer.scala | 35 +++---- ...apViewSerializerSnapshotMigrationTest.java | 47 +++++++++ .../flink-1.6-map-view-serializer-data | Bin 0 -> 440 bytes .../flink-1.6-map-view-serializer-snapshot | Bin 0 -> 1503 bytes 6 files changed, 154 insertions(+), 70 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerSnapshot.java create mode 100644 flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/MapViewSerializerSnapshotMigrationTest.java create mode 100644 flink-libraries/flink-table/src/test/resources/flink-1.6-map-view-serializer-data create mode 100644 flink-libraries/flink-table/src/test/resources/flink-1.6-map-view-serializer-snapshot diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java deleted file mode 100644 index 10b4419a8c7c9..0000000000000 --- a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java +++ /dev/null @@ -1,47 +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.table.dataview; - -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.table.api.dataview.MapView; - -/** - * A {@link TypeSerializerConfigSnapshot} for the {@link MapViewSerializer}. - * - * @param the key type of the map entries. - * @param the value type of the map entries. - */ -public class MapViewSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { - - private static final int VERSION = 1; - - /** This empty nullary constructor is required for deserializing the configuration. */ - public MapViewSerializerConfigSnapshot() {} - - public MapViewSerializerConfigSnapshot(MapSerializer mapSerializer) { - super(mapSerializer); - } - - @Override - public int getVersion() { - return VERSION; - } -} diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerSnapshot.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerSnapshot.java new file mode 100644 index 0000000000000..f59fc0a365423 --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerSnapshot.java @@ -0,0 +1,95 @@ +/* + * 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.table.dataview; + +import org.apache.flink.api.common.typeutils.CompositeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.api.dataview.MapView; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A {@link TypeSerializerSnapshot} for the {@link MapViewSerializer}. + * + * @param the key type of the map entries. + * @param the value type of the map entries. + */ +public class MapViewSerializerSnapshot implements TypeSerializerSnapshot> { + + private static final int CURRENT_VERSION = 1; + + private CompositeSerializerSnapshot nestedMapSerializerSnapshot; + + /** + * Constructor for read instantiation. + */ + public MapViewSerializerSnapshot() {} + + /** + * Constructor to create the snapshot for writing. + */ + public MapViewSerializerSnapshot(TypeSerializer> mapSerializer) { + this.nestedMapSerializerSnapshot = new CompositeSerializerSnapshot(Preconditions.checkNotNull(mapSerializer)); + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public TypeSerializer> restoreSerializer() { + return new MapViewSerializer<>(nestedMapSerializerSnapshot.getRestoreSerializer(0)); + } + + @Override + public TypeSerializerSchemaCompatibility> resolveSchemaCompatibility( + TypeSerializer> newSerializer) { + checkState(nestedMapSerializerSnapshot != null); + + if (newSerializer instanceof MapViewSerializer) { + MapViewSerializer serializer = (MapViewSerializer) newSerializer; + + return nestedMapSerializerSnapshot.resolveCompatibilityWithNested( + TypeSerializerSchemaCompatibility.compatibleAsIs(), + serializer.getMapSerializer()); + } + else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + nestedMapSerializerSnapshot.writeCompositeSnapshot(out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.nestedMapSerializerSnapshot = CompositeSerializerSnapshot.readCompositeSnapshot(in, userCodeClassLoader); + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala index d6419e81ab351..89cdf70174983 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala @@ -35,14 +35,14 @@ import org.apache.flink.table.api.dataview.MapView * @tparam K The type of the keys in the map. * @tparam V The type of the values in the map. */ -class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V]) +@SerialVersionUID(-9007142882049098705L) +class MapViewSerializer[K, V](val mapSerializer: TypeSerializer[java.util.Map[K, V]]) extends TypeSerializer[MapView[K, V]] { override def isImmutableType: Boolean = false override def duplicate(): TypeSerializer[MapView[K, V]] = - new MapViewSerializer[K, V]( - mapSerializer.duplicate().asInstanceOf[MapSerializer[K, V]]) + new MapViewSerializer[K, V](mapSerializer.duplicate()) override def createInstance(): MapView[K, V] = { new MapView[K, V]() @@ -77,30 +77,14 @@ class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V]) override def equals(obj: Any): Boolean = canEqual(this) && mapSerializer.equals(obj.asInstanceOf[MapViewSerializer[_, _]].mapSerializer) - override def snapshotConfiguration(): MapViewSerializerConfigSnapshot[K, V] = - new MapViewSerializerConfigSnapshot[K, V](mapSerializer) + override def snapshotConfiguration(): MapViewSerializerSnapshot[K, V] = + new MapViewSerializerSnapshot[K, V](mapSerializer) // copy and modified from MapSerializer.ensureCompatibility override def ensureCompatibility(configSnapshot: TypeSerializerConfigSnapshot[_]) : CompatibilityResult[MapView[K, V]] = { configSnapshot match { - case snapshot: MapViewSerializerConfigSnapshot[K, V] => - val previousKvSerializersAndConfigs = - snapshot.getNestedSerializersAndConfigs - - val mapSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult( - previousKvSerializersAndConfigs.get(0).f0, - classOf[UnloadableDummyTypeSerializer[_]], - previousKvSerializersAndConfigs.get(0).f1, - mapSerializer) - - if (!mapSerializerCompatResult.isRequiresMigration) { - CompatibilityResult.compatible[MapView[K, V]] - } else { - CompatibilityResult.requiresMigration[MapView[K, V]] - } - // backwards compatibility path; // Flink versions older or equal to 1.5.x returns a // MapSerializerConfigSnapshot as the snapshot @@ -108,17 +92,20 @@ class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V]) val previousKvSerializersAndConfigs = legacySnapshot.getNestedSerializersAndConfigs + // in older versions, the nested map serializer was always + // specifically a MapSerializer, so this cast is safe + val castedSer = mapSerializer.asInstanceOf[MapSerializer[K, V]] val keyCompatResult = CompatibilityUtil.resolveCompatibilityResult( previousKvSerializersAndConfigs.get(0).f0, classOf[UnloadableDummyTypeSerializer[_]], previousKvSerializersAndConfigs.get(0).f1, - mapSerializer) + castedSer.getKeySerializer) val valueCompatResult = CompatibilityUtil.resolveCompatibilityResult( previousKvSerializersAndConfigs.get(1).f0, classOf[UnloadableDummyTypeSerializer[_]], previousKvSerializersAndConfigs.get(1).f1, - mapSerializer.getValueSerializer) + castedSer.getValueSerializer) if (!keyCompatResult.isRequiresMigration && !valueCompatResult.isRequiresMigration) { CompatibilityResult.compatible[MapView[K, V]] @@ -129,4 +116,6 @@ class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V]) case _ => CompatibilityResult.requiresMigration[MapView[K, V]] } } + + def getMapSerializer: TypeSerializer[java.util.Map[K, V]] = mapSerializer } diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/MapViewSerializerSnapshotMigrationTest.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/MapViewSerializerSnapshotMigrationTest.java new file mode 100644 index 0000000000000..66c7f17347456 --- /dev/null +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/dataview/MapViewSerializerSnapshotMigrationTest.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.table.dataview; + +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.table.api.dataview.MapView; + +/** + * Migration test for the {@link MapViewSerializerSnapshot}. + */ +public class MapViewSerializerSnapshotMigrationTest extends TypeSerializerSnapshotMigrationTestBase> { + + private static final String DATA = "flink-1.6-map-view-serializer-data"; + private static final String SNAPSHOT = "flink-1.6-map-view-serializer-snapshot"; + + public MapViewSerializerSnapshotMigrationTest() { + super( + TestSpecification.>builder( + "1.6-map-view-serializer", + MapViewSerializer.class, + MapViewSerializerSnapshot.class) + .withSerializerProvider(() -> new MapViewSerializer<>( + new MapSerializer<>(IntSerializer.INSTANCE, StringSerializer.INSTANCE))) + .withSnapshotDataLocation(SNAPSHOT) + .withTestData(DATA, 10) + ); + } +} diff --git a/flink-libraries/flink-table/src/test/resources/flink-1.6-map-view-serializer-data b/flink-libraries/flink-table/src/test/resources/flink-1.6-map-view-serializer-data new file mode 100644 index 0000000000000000000000000000000000000000..6d68f192c03556b4ac511ae7e24e35fc81fb5112 GIT binary patch literal 440 zcmZQzU|?ZjU|?lnNz2a%GPoF6k`jyl|NrmGz*e4Ll#>D!<7QyXNX^LsNsp=>B;f!6 DoZDra literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-table/src/test/resources/flink-1.6-map-view-serializer-snapshot b/flink-libraries/flink-table/src/test/resources/flink-1.6-map-view-serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..9546a160ec0dcc9f51ada4767485523bb8eff7fc GIT binary patch literal 1503 zcmd6l%}N6?6oqdC1vlcGB#0tncSxbD z+_=zZ@Mfku9dyy5MI4x)B;1d4&Kv+JL2{88p!|fNxF)@acPK{&Rs++fJmk%epe-eJ zms;LXf}U~92T}~_0S`~`?Fl2flIOx)!CG~{cEJdg>agtdFgvoaRYx96c`V`KLb7Jy z`+;UQ4n^Ne6|n}71VeN-TNxQx{~-p4X*x{5hv^Ywvcc3GlnB5mL`R#mL7W0-E(6{6 z`ZveV=8gbCV_gwvM;10_G?v#^sl&>EtNxVS9$Z3sB549$()-(qYxb_@%B=0P9(UsNV%T-cnM$S^;A2=j;T!CEK3nVblq1T4F%0$@PYv z#yxV1_XqB{!oN%kUZf6xm+&IGo*3PoEzm=5xG$_QN<>k<0co-cY@V!K=C6~?7uB1E L7yEz4K2KbqQV9Lq literal 0 HcmV?d00001 From a0720aba4a1ebdb54cf7b713a59f6354b5dd18be Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 6 Nov 2018 14:26:54 +0800 Subject: [PATCH 053/359] [FLINK-10789] [scala] ScalaEitherSerializerSnapshot should be a TypeSerializerSnapshot This also adds a migration test for the Scala EitherSerializer. This closes #7028. --- flink-scala/pom.xml | 1 + .../ScalaEitherSerializerConfigSnapshot.java | 46 -------- .../ScalaEitherSerializerSnapshot.java | 98 ++++++++++++++++++ .../scala/typeutils/EitherSerializer.scala | 11 +- ...EitherSerializerSnapshotMigrationTest.java | 47 +++++++++ .../flink-1.6-scala-either-serializer-data | Bin 0 -> 130 bytes ...flink-1.6-scala-either-serializer-snapshot | Bin 0 -> 1358 bytes 7 files changed, 152 insertions(+), 51 deletions(-) delete mode 100644 flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java create mode 100644 flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshot.java create mode 100644 flink-scala/src/test/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshotMigrationTest.java create mode 100644 flink-scala/src/test/resources/flink-1.6-scala-either-serializer-data create mode 100644 flink-scala/src/test/resources/flink-1.6-scala-either-serializer-snapshot diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index e7fc6e13c2b50..bf1ea5f03b3c4 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -244,6 +244,7 @@ under the License. + src/test/java src/test/scala diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java deleted file mode 100644 index 8ad4e09a1e26d..0000000000000 --- a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java +++ /dev/null @@ -1,46 +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.scala.typeutils; - -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; - -import scala.util.Either; - -/** - * Configuration snapshot for serializers of Scala's {@link Either} type, - * containing configuration snapshots of the Left and Right serializers. - */ -public class ScalaEitherSerializerConfigSnapshot - extends CompositeTypeSerializerConfigSnapshot> { - - private static final int VERSION = 1; - - /** This empty nullary constructor is required for deserializing the configuration. */ - public ScalaEitherSerializerConfigSnapshot() {} - - public ScalaEitherSerializerConfigSnapshot(TypeSerializer leftSerializer, TypeSerializer rightSerializer) { - super(leftSerializer, rightSerializer); - } - - @Override - public int getVersion() { - return VERSION; - } -} diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshot.java b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshot.java new file mode 100644 index 0000000000000..b67e47b4effa2 --- /dev/null +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshot.java @@ -0,0 +1,98 @@ +/* + * 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.scala.typeutils; + +import org.apache.flink.api.common.typeutils.CompositeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +import scala.util.Either; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Configuration snapshot for serializers of Scala's {@link Either} type, + * containing configuration snapshots of the Left and Right serializers. + */ +public class ScalaEitherSerializerSnapshot implements TypeSerializerSnapshot> { + + private static final int CURRENT_VERSION = 1; + + private CompositeSerializerSnapshot nestedLeftRightSerializerSnapshot; + + /** + * Constructor for read instantiation. + */ + public ScalaEitherSerializerSnapshot() {} + + /** + * Constructor to create the snapshot for writing. + */ + public ScalaEitherSerializerSnapshot(TypeSerializer leftSerializer, TypeSerializer rightSerializer) { + Preconditions.checkNotNull(leftSerializer); + Preconditions.checkNotNull(rightSerializer); + this.nestedLeftRightSerializerSnapshot = new CompositeSerializerSnapshot(leftSerializer, rightSerializer); + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public TypeSerializer> restoreSerializer() { + return new EitherSerializer<>( + nestedLeftRightSerializerSnapshot.getRestoreSerializer(0), + nestedLeftRightSerializerSnapshot.getRestoreSerializer(1)); + } + + @Override + public TypeSerializerSchemaCompatibility> resolveSchemaCompatibility( + TypeSerializer> newSerializer) { + checkState(nestedLeftRightSerializerSnapshot != null); + + if (newSerializer instanceof EitherSerializer) { + EitherSerializer serializer = (EitherSerializer) newSerializer; + + return nestedLeftRightSerializerSnapshot.resolveCompatibilityWithNested( + TypeSerializerSchemaCompatibility.compatibleAsIs(), + serializer.getLeftSerializer(), + serializer.getRightSerializer()); + } + else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + nestedLeftRightSerializerSnapshot.writeCompositeSnapshot(out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.nestedLeftRightSerializerSnapshot = CompositeSerializerSnapshot.readCompositeSnapshot(in, userCodeClassLoader); + } +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala index 14f2196b9ccd3..68432a6f1a502 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala @@ -116,21 +116,22 @@ class EitherSerializer[A, B]( 31 * leftSerializer.hashCode() + rightSerializer.hashCode() } + def getLeftSerializer: TypeSerializer[A] = leftSerializer + + def getRightSerializer: TypeSerializer[B] = rightSerializer + // -------------------------------------------------------------------------------------------- // Serializer configuration snapshotting & compatibility // -------------------------------------------------------------------------------------------- - override def snapshotConfiguration(): ScalaEitherSerializerConfigSnapshot[A, B] = { - new ScalaEitherSerializerConfigSnapshot[A, B](leftSerializer, rightSerializer) + override def snapshotConfiguration(): ScalaEitherSerializerSnapshot[A, B] = { + new ScalaEitherSerializerSnapshot[A, B](leftSerializer, rightSerializer) } override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot[_]): CompatibilityResult[Either[A, B]] = { configSnapshot match { - case eitherSerializerConfig: ScalaEitherSerializerConfigSnapshot[A, B] => - checkCompatibility(eitherSerializerConfig) - // backwards compatibility path; // Flink versions older or equal to 1.5.x uses a // EitherSerializerConfigSnapshot as the snapshot diff --git a/flink-scala/src/test/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshotMigrationTest.java b/flink-scala/src/test/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshotMigrationTest.java new file mode 100644 index 0000000000000..9cd8b5d55b5dd --- /dev/null +++ b/flink-scala/src/test/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerSnapshotMigrationTest.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.api.scala.typeutils; + +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; + +import scala.util.Either; + +/** + * Migration test for the {@link ScalaEitherSerializerSnapshot}. + */ +public class ScalaEitherSerializerSnapshotMigrationTest extends TypeSerializerSnapshotMigrationTestBase> { + + private static final String DATA = "flink-1.6-scala-either-serializer-data"; + private static final String SNAPSHOT = "flink-1.6-scala-either-serializer-snapshot"; + + public ScalaEitherSerializerSnapshotMigrationTest() { + super( + TestSpecification.>builder( + "1.6-scala-either-serializer", + EitherSerializer.class, + ScalaEitherSerializerSnapshot.class) + .withSerializerProvider(() -> new EitherSerializer<>(IntSerializer.INSTANCE, StringSerializer.INSTANCE)) + .withSnapshotDataLocation(SNAPSHOT) + .withTestData(DATA, 10) + ); + } + +} diff --git a/flink-scala/src/test/resources/flink-1.6-scala-either-serializer-data b/flink-scala/src/test/resources/flink-1.6-scala-either-serializer-data new file mode 100644 index 0000000000000000000000000000000000000000..203067c5a04a3d1365646a61f470a3bc44d5e9a6 GIT binary patch literal 130 XcmZSJ$w{BJdg%J#4`4%LY-ev4b_N6uDrn%KREQwT zvn*G^A2m27{_J0G0GnDQX$^G+ci@dQY`2Q6cr}$xUez>sKESA;PIvzYIp+r1k*cYv zgEZ_ffN2T}g-RsaT#cU}U*6bFzf@MX=##qM0@%=MLd~433oLmZH{;`0;sw914^|JA z6&cPA_#juYNJ7O6ljhd(07$P}pOhcE^Ur91(@mMLk7=IP)8$n}$TfP{YU*4PPq^k8 hTGSGaX{MlZ*1?wXs`Pv^^?X&?S$VkE)AnWT`UTT)${qj! literal 0 HcmV?d00001 From 29e60a51e6c5006179e3b00f7c46f4bde55a4cff Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 6 Nov 2018 14:28:28 +0800 Subject: [PATCH 054/359] [hotfix] [core] Add missing precondition null checks in List-/MapSerializerSnapshot --- .../api/common/typeutils/base/ListSerializerSnapshot.java | 3 ++- .../flink/api/common/typeutils/base/MapSerializerSnapshot.java | 3 +++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerSnapshot.java index 2caf9fa18c617..5f89d94973f81 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerSnapshot.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerSnapshot.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; import java.io.IOException; import java.util.List; @@ -48,7 +49,7 @@ public ListSerializerSnapshot() {} * Constructor to create the snapshot for writing. */ public ListSerializerSnapshot(TypeSerializer elementSerializer) { - this.nestedElementSerializerSnapshot = new CompositeSerializerSnapshot(elementSerializer); + this.nestedElementSerializerSnapshot = new CompositeSerializerSnapshot(Preconditions.checkNotNull(elementSerializer)); } @Override diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerSnapshot.java index 92d17996ea5c1..be2e4b0cbe6a0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerSnapshot.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerSnapshot.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; import java.io.IOException; import java.util.Map; @@ -48,6 +49,8 @@ public MapSerializerSnapshot() {} * Constructor to create the snapshot for writing. */ public MapSerializerSnapshot(TypeSerializer keySerializer, TypeSerializer valueSerializer) { + Preconditions.checkNotNull(keySerializer); + Preconditions.checkNotNull(valueSerializer); this.nestedKeyValueSerializerSnapshot = new CompositeSerializerSnapshot(keySerializer, valueSerializer); } From 0cdc1c5a51e5b263835de06e90f215eff0c51c92 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 11 Nov 2018 15:46:04 +0800 Subject: [PATCH 055/359] [hotfix] [cep] Remove duplicate getElementSerializer() method from LockableTypeSerializer --- .../java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java index f7bc968ce0922..7afbc508230c7 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java @@ -207,10 +207,6 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfig : CompatibilityResult.compatible(); } - TypeSerializer getElementSerializer() { - return elementSerializer; - } - @VisibleForTesting TypeSerializer getElementSerializer() { return elementSerializer; From 302f25fa57e4112fe91fe597b984762fb39f5eef Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 11 Nov 2018 14:38:15 +0100 Subject: [PATCH 056/359] [hotfix] Make test_sql_client independent of unzip Some platforms might not have installed the unzip command. Therefore it is better to use the jar command to extract jar files. --- flink-end-to-end-tests/test-scripts/test_sql_client.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index 313ee4436294d..b9d0ecaa0c64a 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -36,7 +36,7 @@ mkdir -p $EXTRACTED_JAR for SQL_JAR in $SQL_JARS_DIR/*.jar; do echo "Checking SQL JAR: $SQL_JAR" - unzip $SQL_JAR -d $EXTRACTED_JAR > /dev/null + (cd $EXTRACTED_JAR && jar xf $SQL_JAR) # check for proper shading for EXTRACTED_FILE in $(find $EXTRACTED_JAR -type f); do @@ -60,6 +60,8 @@ for SQL_JAR in $SQL_JARS_DIR/*.jar; do rm -r $EXTRACTED_JAR/* done +rm -r $EXTRACTED_JAR + ################################################################################ # Prepare connectors ################################################################################ From 096918c6577b06bb9ac6250063c6c99a04907f77 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 8 Nov 2018 17:32:35 +0100 Subject: [PATCH 057/359] [FLINK-10626] [docs] [table] Add documentation for temporal table joins This closes #7065. --- docs/dev/table/streaming/joins.md | 33 +++++++++++++++++++++++++------ 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/docs/dev/table/streaming/joins.md b/docs/dev/table/streaming/joins.md index 24c33b9be6306..f2934066c07c5 100644 --- a/docs/dev/table/streaming/joins.md +++ b/docs/dev/table/streaming/joins.md @@ -136,14 +136,14 @@ SELECT o.amount * r.rate AS amount FROM Orders AS o, - LATERAL TABLE (Rates(o.proctime)) AS r + LATERAL TABLE (Rates(o.rowtime)) AS r WHERE r.currency = o.currency {% endhighlight %} -Each record from the probe side will be joined with the version of the build side table at the time of the correlated time attribute of the probe side record. +Each record from the probe side will be joined with the version of the build side table at the time of the correlated time attribute of the probe side record. In order to support updates (overwrites) of previous values on the build side table, the table must define a primary key. -In our example, each record from `Orders` will be joined with the version of `Rates` at time `o.proctime`. Because the time attribute defines a processing-time notion, a newly appended order is always joined with the most recent version of `Rates` when executing the operation. The `currency` field has been defined as the primary key of `Rates` before and is used to connect both tables in our example. +In our example, each record from `Orders` will be joined with the version of `Rates` at time `o.rowtime`. The `currency` field has been defined as the primary key of `Rates` before and is used to connect both tables in our example. If the query were using a processing-time notion, a newly appended order would always be joined with the most recent version of `Rates` when executing the operation. In contrast to [regular joins](#regular-joins), this means that if there is a new record on the build side, it will not affect the previous results of the join. This again allows Flink to limit the number of elements that must be kept in the state. @@ -189,14 +189,35 @@ val result = orders +**Note**: State retention defined in a [query configuration](query_configuration.html) is not yet implemented for temporal joins. +This means that the required state to compute the query result might grow infinitely depending on the number of distinct primary keys for the history table. + ### Processing-time Temporal Joins With a processing-time time attribute, it is impossible to pass _past_ time attributes as an argument to the temporal table function. -By definition, it is always the current timestamp. Thus, processing-time temporal table function invocations will always return the latest known versions of the underlying table +By definition, it is always the current timestamp. Thus, invocations of a processing-time temporal table function will always return the latest known versions of the underlying table and any updates in the underlying history table will also immediately overwrite the current values. -Only the latest versions (with respect to the defined primary key) of the build side records are kept in the state. New updates will have no effect on the previously results emitted/processed records from the probe side. +Only the latest versions (with respect to the defined primary key) of the build side records are kept in the state. +New updates will have no effect on the previously results emitted/processed records from the probe side. -One can think about processing-time temporal join as a simple `HashMap` that stores all of the records from the build side. +One can think about a processing-time temporal join as a simple `HashMap` that stores all of the records from the build side. When a new record from the build side has the same key as some previous record, the old value is just simply overwritten. Every record from the probe side is always evaluated against the most recent/current state of the `HashMap`. + +### Event-time Temporal Joins + +With an event-time time attribute (i.e., a rowtime attribute), it is possible to pass _past_ time attributes to the temporal table function. +This allows for joining the two tables at a common point in time. + +Compared to processing-time temporal joins, the temporal table does not only keep the latest version (with respect to the defined primary key) of the build side records in the state +but stores all versions (identified by time) since the last watermark. + +For example, an incoming row with an event-time timestamp of `12:30:00` that is appended to the probe side table +is joined with the version of the build side table at time `12:30:00` according to the [concept of temporal tables](temporal_tables.html). +Thus, the incoming row is only joined with rows that have a timestamp lower or equal to `12:30:00` with +applied updates according to the primary key until this point in time. + +By definition of event time, [watermarks]({{ site.baseurl }}/dev/event_time.html) allow the join operation to move +forward in time and discard versions of the build table that are no longer necessary because no incoming row with +lower or equal timestamp is expected. From 7d9fcbaf1408b09ee1d409ec1568a5af51d726d0 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Thu, 8 Nov 2018 15:08:35 +0100 Subject: [PATCH 058/359] [FLINK-10826] [e2e] Decrease deployment size of heavy deplyment e2e test for Travis This closes #7066. --- .../test-scripts/test_heavy_deployment.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh index 895e4a7f3027e..b4646fc326029 100755 --- a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh +++ b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh @@ -25,7 +25,7 @@ TEST=flink-heavy-deployment-stress-test TEST_PROGRAM_NAME=HeavyDeploymentStressTestProgram TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar -set_conf "taskmanager.heap.mb" "256" # 256Mb x 20TMs = 5Gb total heap +set_conf "taskmanager.heap.mb" "512" # 512Mb x 10TMs = 5Gb total heap set_conf "taskmanager.memory.size" "8" # 8Mb set_conf "taskmanager.network.memory.min" "8mb" @@ -35,12 +35,12 @@ set_conf "taskmanager.memory.segment-size" "8kb" set_conf "taskmanager.numberOfTaskSlots" "10" # 10 slots per TM start_cluster # this also starts 1TM -start_taskmanagers 19 # 1TM + 19TM = 20TM a 10 slots = 200 slots +start_taskmanagers 9 # 1TM + 9TM = 10TM a 10 slots = 100 slots -# This call will result in a deployment with state meta data of 200 x 200 x 50 union states x each 75 entries. +# This call will result in a deployment with state meta data of 100 x 100 x 50 union states x each 100 entries. # We can scale up the numbers to make the test even heavier. $FLINK_DIR/bin/flink run ${TEST_PROGRAM_JAR} \ ---environment.max_parallelism 1024 --environment.parallelism 200 \ +--environment.max_parallelism 1024 --environment.parallelism 100 \ --environment.restart_strategy fixed_delay --environment.restart_strategy.fixed_delay.attempts 3 \ --state_backend.checkpoint_directory ${CHECKPOINT_DIR} \ ---heavy_deployment_test.num_list_states_per_op 50 --heavy_deployment_test.num_partitions_per_list_state 75 +--heavy_deployment_test.num_list_states_per_op 50 --heavy_deployment_test.num_partitions_per_list_state 100 From 78ea3923782fddfd4fbb414c754d4db7701e24ca Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 12 Nov 2018 10:28:51 +0100 Subject: [PATCH 059/359] [FLINK-10826] [e2e] Increase network timeout --- flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh index b4646fc326029..86134709d6867 100755 --- a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh +++ b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh @@ -30,6 +30,7 @@ set_conf "taskmanager.heap.mb" "512" # 512Mb x 10TMs = 5Gb total heap set_conf "taskmanager.memory.size" "8" # 8Mb set_conf "taskmanager.network.memory.min" "8mb" set_conf "taskmanager.network.memory.max" "8mb" +set_conf "taskmanager.network.request-backoff.max" "60000" set_conf "taskmanager.memory.segment-size" "8kb" set_conf "taskmanager.numberOfTaskSlots" "10" # 10 slots per TM From 7dc4c70e2f749f0b0888940cbb1c43d93747d677 Mon Sep 17 00:00:00 2001 From: Igal Shilman Date: Fri, 9 Nov 2018 13:59:24 +0100 Subject: [PATCH 060/359] [FLINK-10821] E2E now uses externalized checkpoint This commit fixes the test_resume_externalized_checkpoints.sh script, by providing the path to the externalized checkpoint taken. --- .../test_resume_externalized_checkpoints.sh | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh index e1ba65d70c3b1..fe2319063b54f 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh @@ -66,8 +66,9 @@ TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-datastream-allround-test/target/DataStr function buildBaseJobCmd { local dop=$1 + local flink_args="$2" - echo "$FLINK_DIR/bin/flink run -d -p $dop $TEST_PROGRAM_JAR \ + echo "$FLINK_DIR/bin/flink run -d ${flink_args} -p $dop $TEST_PROGRAM_JAR \ --test.semantics exactly-once \ --environment.parallelism $dop \ --environment.externalize_checkpoint true \ @@ -125,9 +126,20 @@ fi echo "Restoring job with externalized checkpoint at $CHECKPOINT_PATH ..." -BASE_JOB_CMD=`buildBaseJobCmd $NEW_DOP` +BASE_JOB_CMD=`buildBaseJobCmd $NEW_DOP "-s file://${CHECKPOINT_PATH}"` +JOB_CMD="" +if [[ $SIMULATE_FAILURE == "true" ]]; then + JOB_CMD="$BASE_JOB_CMD \ + --test.simulate_failure true \ + --test.simulate_failure.num_records 0 \ + --test.simulate_failure.num_checkpoints 0 \ + --test.simulate_failure.max_failures 0 \ + --environment.restart_strategy no_restart" +else + JOB_CMD=$BASE_JOB_CMD +fi -DATASTREAM_JOB=$($BASE_JOB_CMD | grep "Job has been submitted with JobID" | sed 's/.* //g') +DATASTREAM_JOB=$($JOB_CMD | grep "Job has been submitted with JobID" | sed 's/.* //g') wait_job_running $DATASTREAM_JOB wait_oper_metric_num_in_records SemanticsCheckMapper.0 200 From 19c34baa6fd9554048108b58baa517addf483c2c Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 9 Nov 2018 12:56:02 +0100 Subject: [PATCH 061/359] [FLINK-10839][serializer] Fix implementation of PojoSerializer.duplicate() w.r.t. subclass serializer --- .../typeutils/runtime/PojoSerializer.java | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java index a4d3839f66f10..f1dd8fcbde952 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java @@ -18,29 +18,17 @@ package org.apache.flink.api.java.typeutils.runtime; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.lang.reflect.Field; -import java.lang.reflect.Modifier; -import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Objects; - import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil; import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -52,6 +40,18 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.util.Preconditions; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Objects; + import static org.apache.flink.util.Preconditions.checkNotNull; @Internal @@ -181,11 +181,13 @@ public PojoSerializer duplicate() { } } - if (stateful) { - return new PojoSerializer(clazz, duplicateFieldSerializers, fields, executionConfig); - } else { - return this; + if (!stateful) { + // as a small memory optimization, we can share the same object between instances + duplicateFieldSerializers = fieldSerializers; } + + // we must create a new instance, otherwise the subclassSerializerCache can create concurrency problems + return new PojoSerializer<>(clazz, duplicateFieldSerializers, fields, executionConfig); } From bd75a06c7aecd3b3f2d6fcf64bd4dde4624828a6 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Thu, 8 Nov 2018 14:44:08 +0100 Subject: [PATCH 062/359] [FLINK-10827][tests] Add test for duplicate() to SerializerTestBase This closes #7061. --- .../common/typeutils/SerializerTestBase.java | 128 +++++++++++++++--- 1 file changed, 112 insertions(+), 16 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java index a4908f95e90c2..151dafb2f8413 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java @@ -18,32 +18,39 @@ package org.apache.flink.api.common.typeutils; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.Arrays; - import org.apache.flink.api.java.typeutils.runtime.NullableSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; -import org.junit.Assert; import org.apache.commons.lang3.SerializationException; import org.apache.commons.lang3.SerializationUtils; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; +import org.junit.Assert; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CyclicBarrier; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Abstract test base for serializers. * @@ -436,6 +443,33 @@ public void testNullability() { } } + @Test + public void testDuplicate() throws Exception { + final int numThreads = 10; + final TypeSerializer serializer = getSerializer(); + final CyclicBarrier startLatch = new CyclicBarrier(numThreads); + final List> concurrentRunners = new ArrayList<>(numThreads); + Assert.assertEquals(serializer, serializer.duplicate()); + + T[] testData = getData(); + + for (int i = 0; i < numThreads; ++i) { + SerializerRunner runner = new SerializerRunner<>( + startLatch, + serializer.duplicate(), + testData, + 120L); + + runner.start(); + concurrentRunners.add(runner); + } + + for (SerializerRunner concurrentRunner : concurrentRunners) { + concurrentRunner.join(); + concurrentRunner.checkResult(); + } + } + // -------------------------------------------------------------------------------------------- protected void deepEquals(String message, T should, T is) { @@ -526,6 +560,68 @@ public void write(DataInputView source, int numBytes) throws IOException { } } + /** + * Runner to test serializer duplication via concurrency. + * @param type of the test elements. + */ + static class SerializerRunner extends Thread { + final CyclicBarrier allReadyBarrier; + final TypeSerializer serializer; + final T[] testData; + final long durationLimitMillis; + Exception failure; + + SerializerRunner( + CyclicBarrier allReadyBarrier, + TypeSerializer serializer, + T[] testData, + long testTargetDurationMillis) { + + this.allReadyBarrier = allReadyBarrier; + this.serializer = serializer; + this.testData = testData; + this.durationLimitMillis = testTargetDurationMillis; + this.failure = null; + } + + @Override + public void run() { + DataInputDeserializer dataInputDeserializer = new DataInputDeserializer(); + DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(128); + try { + allReadyBarrier.await(); + final long endTimeNanos = System.nanoTime() + durationLimitMillis * 1_000_000L; + while (true) { + for (T testItem : testData) { + serializer.serialize(testItem, dataOutputSerializer); + dataInputDeserializer.setBuffer( + dataOutputSerializer.getSharedBuffer(), + 0, + dataOutputSerializer.length()); + T serdeTestItem = serializer.deserialize(dataInputDeserializer); + T copySerdeTestItem = serializer.copy(serdeTestItem); + dataOutputSerializer.clear(); + + Preconditions.checkState(Objects.deepEquals(testItem, copySerdeTestItem), + "Serialization/Deserialization cycle resulted in an object that are not equal to the original."); + + // try to enforce some upper bound to the test time + if (System.nanoTime() >= endTimeNanos) { + return; + } + } + } + } catch (Exception ex) { + failure = ex; + } + } + + void checkResult() throws Exception { + if (failure != null) { + throw failure; + } + } + } private static final class TestInputView extends DataInputStream implements DataInputView { From 45ad36fd752f53b1fa17d7226d3c93614fd24b3f Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 7 Nov 2018 14:05:07 +0100 Subject: [PATCH 063/359] [FLINK-10809][state] Include keyed state that is not from head operators in state assignment This closes #7048. Signed-off-by: Stefan Richter --- .../checkpoint/StateAssignmentOperation.java | 29 ++---- ...nterpretDataStreamAsKeyedStreamITCase.java | 99 +++++++++++++++++-- 2 files changed, 102 insertions(+), 26 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java index b0173886d5730..02fc2013fb049 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java @@ -71,13 +71,12 @@ public StateAssignmentOperation( this.allowNonRestoredState = allowNonRestoredState; } - public boolean assignStates() throws Exception { + public void assignStates() { Map localOperators = new HashMap<>(operatorStates); - Map localTasks = this.tasks; checkStateMappingCompleteness(allowNonRestoredState, operatorStates, tasks); - for (Map.Entry task : localTasks.entrySet()) { + for (Map.Entry task : this.tasks.entrySet()) { final ExecutionJobVertex executionJobVertex = task.getValue(); // find the states of all operators belonging to this task @@ -108,7 +107,6 @@ public boolean assignStates() throws Exception { assignAttemptState(task.getValue(), operatorStates); } - return true; } private void assignAttemptState(ExecutionJobVertex executionJobVertex, List operatorStates) { @@ -254,10 +252,6 @@ public static OperatorSubtaskState operatorSubtaskStateFrom( new StateObjectCollection<>(subRawKeyedState.getOrDefault(instanceID, Collections.emptyList()))); } - private static boolean isHeadOperator(int opIdx, List operatorIDs) { - return opIdx == operatorIDs.size() - 1; - } - public void checkParallelismPreconditions(List operatorStates, ExecutionJobVertex executionJobVertex) { for (OperatorState operatorState : operatorStates) { checkParallelismPreconditions(operatorState, executionJobVertex); @@ -278,19 +272,16 @@ private void reDistributeKeyedStates( for (int operatorIndex = 0; operatorIndex < newOperatorIDs.size(); operatorIndex++) { OperatorState operatorState = oldOperatorStates.get(operatorIndex); int oldParallelism = operatorState.getParallelism(); - for (int subTaskIndex = 0; subTaskIndex < newParallelism; subTaskIndex++) { OperatorInstanceID instanceID = OperatorInstanceID.of(subTaskIndex, newOperatorIDs.get(operatorIndex)); - if (isHeadOperator(operatorIndex, newOperatorIDs)) { - Tuple2, List> subKeyedStates = reAssignSubKeyedStates( - operatorState, - newKeyGroupPartitions, - subTaskIndex, - newParallelism, - oldParallelism); - newManagedKeyedState.put(instanceID, subKeyedStates.f0); - newRawKeyedState.put(instanceID, subKeyedStates.f1); - } + Tuple2, List> subKeyedStates = reAssignSubKeyedStates( + operatorState, + newKeyGroupPartitions, + subTaskIndex, + newParallelism, + oldParallelism); + newManagedKeyedState.put(instanceID, subKeyedStates.f0); + newRawKeyedState.put(instanceID, subKeyedStates.f1); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java index fc8e9971683f9..6a1b9ee75f7ef 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java @@ -18,11 +18,18 @@ package org.apache.flink.streaming.api.datastream; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.CheckpointListener; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; @@ -73,6 +80,8 @@ public void testReinterpretAsKeyedStream() throws Exception { env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); env.setMaxParallelism(maxParallelism); env.setParallelism(parallelism); + env.enableCheckpointing(100); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0L)); final List partitionFiles = new ArrayList<>(parallelism); for (int i = 0; i < parallelism; ++i) { @@ -156,15 +165,22 @@ public void invoke(Tuple2 value, Context context) throws Excep } } - private static class FromPartitionFileSource extends RichParallelSourceFunction> { + private static class FromPartitionFileSource extends RichParallelSourceFunction> + implements CheckpointedFunction, CheckpointListener { private static final long serialVersionUID = 1L; private List allPartitions; private DataInputStream din; private volatile boolean running; - public FromPartitionFileSource(List allPartitons) { - this.allPartitions = allPartitons; + private long position; + private transient ListState positionState; + private transient boolean isRestored; + + private transient volatile boolean canFail; + + public FromPartitionFileSource(List allPartitions) { + this.allPartitions = allPartitions; } @Override @@ -174,6 +190,11 @@ public void open(Configuration parameters) throws Exception { din = new DataInputStream( new BufferedInputStream( new FileInputStream(allPartitions.get(subtaskIdx)))); + + long toSkip = position; + while (toSkip > 0L) { + toSkip -= din.skip(toSkip); + } } @Override @@ -187,11 +208,29 @@ public void run(SourceContext> out) throws Exception { this.running = true; try { while (running) { - Integer key = din.readInt(); - Integer val = din.readInt(); - out.collect(new Tuple2<>(key, val)); + + checkFail(); + + synchronized (out.getCheckpointLock()) { + Integer key = din.readInt(); + Integer val = din.readInt(); + out.collect(new Tuple2<>(key, val)); + + position += 2 * Integer.BYTES; + } } } catch (EOFException ignore) { + if (!isRestored) { + while (true) { + checkFail(); + } + } + } + } + + private void checkFail() throws Exception { + if (canFail) { + throw new Exception("Artificial failure."); } } @@ -199,14 +238,43 @@ public void run(SourceContext> out) throws Exception { public void cancel() { this.running = false; } + + @Override + public void notifyCheckpointComplete(long checkpointId) { + canFail = !isRestored; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + positionState.add(position); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + canFail = false; + position = 0L; + isRestored = context.isRestored(); + positionState = context.getOperatorStateStore().getListState( + new ListStateDescriptor<>("posState", Long.class)); + + if (isRestored) { + + for (Long value : positionState.get()) { + position += value; + } + } + } } - private static class ValidatingSink extends RichSinkFunction> { + private static class ValidatingSink extends RichSinkFunction> + implements CheckpointedFunction { private static final long serialVersionUID = 1L; private final int expectedSum; private int runningSum = 0; + private transient ListState sumState; + private ValidatingSink(int expectedSum) { this.expectedSum = expectedSum; } @@ -227,5 +295,22 @@ public void close() throws Exception { Assert.assertEquals(expectedSum, runningSum); super.close(); } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + sumState.add(runningSum); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + sumState = context.getOperatorStateStore().getListState( + new ListStateDescriptor<>("sumState", Integer.class)); + + if (context.isRestored()) { + for (Integer value : sumState.get()) { + runningSum += value; + } + } + } } } From 9477b6a9b8a640d70079543329f7de844403f88d Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Thu, 8 Nov 2018 15:51:16 +0100 Subject: [PATCH 064/359] [FLINK-10753] Improve propagation and logging of snapshot exceptions This closes #7064. Signed-off-by: Stefan Richter --- .../checkpoint/CheckpointCoordinator.java | 9 +++++--- .../runtime/checkpoint/PendingCheckpoint.java | 21 +++++++++---------- .../rpc/RpcCheckpointResponder.java | 6 ++++++ .../api/operators/AbstractStreamOperator.java | 6 ++++-- .../AbstractOperatorRestoreTestBase.java | 1 + 5 files changed, 27 insertions(+), 16 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 57337b6286fb1..02b6fa4a2bb99 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -1249,11 +1249,14 @@ private void discardCheckpoint(PendingCheckpoint pendingCheckpoint, @Nullable Th final long checkpointId = pendingCheckpoint.getCheckpointId(); - final String reason = (cause != null) ? cause.getMessage() : ""; + LOG.info("Discarding checkpoint {} of job {}.", checkpointId, job, cause); - LOG.info("Discarding checkpoint {} of job {} because: {}", checkpointId, job, reason); + if (cause != null) { + pendingCheckpoint.abortError(cause); + } else { + pendingCheckpoint.abortDeclined(); + } - pendingCheckpoint.abortDeclined(); rememberRecentCheckpointId(checkpointId); // we don't have to schedule another "dissolving" checkpoint any more because the diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 1b51ac4bf8d3a..1bc6b0e4baa9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; @@ -433,25 +434,23 @@ public void abortSubsumed() { } } + public void abortDeclined() { - try { - Exception cause = new Exception("Checkpoint was declined (tasks not ready)"); - onCompletionPromise.completeExceptionally(cause); - reportFailedCheckpoint(cause); - } finally { - dispose(true); - } + abortWithCause(new Exception("Checkpoint was declined (tasks not ready)")); } /** * Aborts the pending checkpoint due to an error. * @param cause The error's exception. */ - public void abortError(Throwable cause) { + public void abortError(@Nonnull Throwable cause) { + abortWithCause(new Exception("Checkpoint failed: " + cause.getMessage(), cause)); + } + + private void abortWithCause(@Nonnull Exception cause) { try { - Exception failure = new Exception("Checkpoint failed: " + cause.getMessage(), cause); - onCompletionPromise.completeExceptionally(failure); - reportFailedCheckpoint(failure); + onCompletionPromise.completeExceptionally(cause); + reportFailedCheckpoint(cause); } finally { dispose(true); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java index aba8bda191825..918fa50483de5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java @@ -26,8 +26,13 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class RpcCheckpointResponder implements CheckpointResponder { + private static final Logger LOG = LoggerFactory.getLogger(RpcCheckpointResponder.class); + private final CheckpointCoordinatorGateway checkpointCoordinatorGateway; public RpcCheckpointResponder(CheckpointCoordinatorGateway checkpointCoordinatorGateway) { @@ -57,6 +62,7 @@ public void declineCheckpoint( long checkpointId, Throwable cause) { + LOG.info("Declining checkpoint {} of job {}.", checkpointId, jobID, cause); checkpointCoordinatorGateway.declineCheckpoint(jobID, executionAttemptID, checkpointId, cause); } } 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 a63a7971679d6..4967cb9ead9c0 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 @@ -413,8 +413,10 @@ public final OperatorSnapshotFutures snapshotState(long checkpointId, long times snapshotException.addSuppressed(e); } - throw new Exception("Could not complete snapshot " + checkpointId + " for operator " + - getOperatorName() + '.', snapshotException); + String snapshotFailMessage = "Could not complete snapshot " + checkpointId + " for operator " + + getOperatorName() + "."; + + throw new Exception(snapshotFailMessage, snapshotException); } return snapshotInProgress; diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index 3db0f62f8292a..097616feb9663 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -142,6 +142,7 @@ private String migrateJob(ClassLoader classLoader, ClusterClient clusterClien } catch (Exception e) { String exceptionString = ExceptionUtils.stringifyException(e); if (!(exceptionString.matches("(.*\n)*.*savepoint for the job .* failed(.*\n)*") // legacy + || exceptionString.matches("(.*\n)*.*was not running(.*\n)*") || exceptionString.matches("(.*\n)*.*Not all required tasks are currently running(.*\n)*") // new || exceptionString.matches("(.*\n)*.*Checkpoint was declined \\(tasks not ready\\)(.*\n)*"))) { // new throw e; From eb5ff6dada90b63cdeef6a84ba87473a762af7ff Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 13 Nov 2018 09:46:31 +0100 Subject: [PATCH 065/359] [FLINK-10826] [e2e] Further decrease heavy deployment size In order to enable stable nightly Travis cron jobs. --- flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh index 86134709d6867..cc1a64d9623ef 100755 --- a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh +++ b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh @@ -38,10 +38,10 @@ set_conf "taskmanager.numberOfTaskSlots" "10" # 10 slots per TM start_cluster # this also starts 1TM start_taskmanagers 9 # 1TM + 9TM = 10TM a 10 slots = 100 slots -# This call will result in a deployment with state meta data of 100 x 100 x 50 union states x each 100 entries. +# This call will result in a deployment with state meta data of 100 x 100 x 50 union states x each 50 entries. # We can scale up the numbers to make the test even heavier. $FLINK_DIR/bin/flink run ${TEST_PROGRAM_JAR} \ --environment.max_parallelism 1024 --environment.parallelism 100 \ --environment.restart_strategy fixed_delay --environment.restart_strategy.fixed_delay.attempts 3 \ --state_backend.checkpoint_directory ${CHECKPOINT_DIR} \ ---heavy_deployment_test.num_list_states_per_op 50 --heavy_deployment_test.num_partitions_per_list_state 100 +--heavy_deployment_test.num_list_states_per_op 50 --heavy_deployment_test.num_partitions_per_list_state 50 From eb2d9429909e03f32f2f193acf13eb1f05893042 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 13 Nov 2018 09:31:45 +0100 Subject: [PATCH 066/359] [hotfix] CheckpointDeclineException should lead to pendingCheckpoint.abortDecline() We also avoid logging exceptions that are cause by instances of CheckpointDeclineException --- flink-end-to-end-tests/test-scripts/common.sh | 1 + .../flink/runtime/checkpoint/CheckpointCoordinator.java | 7 ++++--- .../runtime/taskexecutor/rpc/RpcCheckpointResponder.java | 8 +------- .../streaming/api/operators/AbstractStreamOperator.java | 1 + 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 41dfe0fc4cbbe..0024d82f3d65a 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -355,6 +355,7 @@ function check_logs_for_exceptions { | grep -v "java.io.InvalidClassException: org.apache.flink.formats.avro.typeutils.AvroSerializer" \ | grep -v "Caused by: java.lang.Exception: JobManager is shutting down" \ | grep -v "java.lang.Exception: Artificial failure" \ + | grep -v "org.apache.flink.runtime.checkpoint.decline" \ | grep -ic "exception" || true) if [[ ${exception_count} -gt 0 ]]; then echo "Found exception in log files:" diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 02b6fa4a2bb99..3890f79ce7d3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineException; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.ExecutionState; @@ -1251,10 +1252,10 @@ private void discardCheckpoint(PendingCheckpoint pendingCheckpoint, @Nullable Th LOG.info("Discarding checkpoint {} of job {}.", checkpointId, job, cause); - if (cause != null) { - pendingCheckpoint.abortError(cause); - } else { + if (cause == null || cause instanceof CheckpointDeclineException) { pendingCheckpoint.abortDeclined(); + } else { + pendingCheckpoint.abortError(cause); } rememberRecentCheckpointId(checkpointId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java index 918fa50483de5..c8f7357ab7e18 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java @@ -26,13 +26,8 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public class RpcCheckpointResponder implements CheckpointResponder { - private static final Logger LOG = LoggerFactory.getLogger(RpcCheckpointResponder.class); - private final CheckpointCoordinatorGateway checkpointCoordinatorGateway; public RpcCheckpointResponder(CheckpointCoordinatorGateway checkpointCoordinatorGateway) { @@ -59,10 +54,9 @@ public void acknowledgeCheckpoint( public void declineCheckpoint( JobID jobID, ExecutionAttemptID executionAttemptID, - long checkpointId, + long checkpointId, Throwable cause) { - LOG.info("Declining checkpoint {} of job {}.", checkpointId, jobID, cause); checkpointCoordinatorGateway.declineCheckpoint(jobID, executionAttemptID, checkpointId, cause); } } 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 4967cb9ead9c0..04489b0ffd986 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 @@ -416,6 +416,7 @@ public final OperatorSnapshotFutures snapshotState(long checkpointId, long times String snapshotFailMessage = "Could not complete snapshot " + checkpointId + " for operator " + getOperatorName() + "."; + LOG.info(snapshotFailMessage, snapshotException); throw new Exception(snapshotFailMessage, snapshotException); } From 44777b6e74e6a393b4ce6dd9b984e6547bf2f29c Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 13 Nov 2018 12:32:56 +0100 Subject: [PATCH 067/359] [FLINK-10863][tests] Assign UIDs to all operators in DataStreamAllroundTestProgram This closes #7085. --- .../tests/DataStreamAllroundTestProgram.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java index b14e2af1b52bd..3c406c7598d5c 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java @@ -78,7 +78,7 @@ public static void main(String[] args) throws Exception { setupEnvironment(env, pt); // add a keyed stateful map operator, which uses Kryo for state serialization - DataStream eventStream = env.addSource(createEventSource(pt)) + DataStream eventStream = env.addSource(createEventSource(pt)).uid("0001") .assignTimestampsAndWatermarks(createTimestampExtractor(pt)) .keyBy(Event::getKey) .map(createArtificialKeyedStateMapper( @@ -97,7 +97,7 @@ public static void main(String[] args) throws Exception { new StatefulComplexPayloadSerializer()), // custom stateful serializer Collections.singletonList(ComplexPayload.class) // KryoSerializer via type extraction ) - ).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Kryo_and_Custom_Stateful"); + ).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Kryo_and_Custom_Stateful").uid("0002"); // add a keyed stateful map operator, which uses Avro for state serialization eventStream = eventStream @@ -124,12 +124,12 @@ public static void main(String[] args) throws Exception { new AvroSerializer<>(ComplexPayloadAvro.class)), // custom AvroSerializer Collections.singletonList(ComplexPayloadAvro.class) // AvroSerializer via type extraction ) - ).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Avro"); + ).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Avro").uid("0003"); DataStream eventStream2 = eventStream .map(createArtificialOperatorStateMapper((MapFunction) in -> in)) - .name(OPERATOR_STATE_OPER_NAME) - .returns(Event.class); + .returns(Event.class) + .name(OPERATOR_STATE_OPER_NAME).uid("0004"); // apply a tumbling window that simply passes forward window elements; // this allows the job to cover timers state @@ -141,19 +141,20 @@ public void apply(Integer integer, TimeWindow window, Iterable input, Col out.collect(e); } } - }).name(TIME_WINDOW_OPER_NAME); + }).name(TIME_WINDOW_OPER_NAME).uid("0005"); if (isSimulateFailures(pt)) { eventStream3 = eventStream3 .map(createFailureMapper(pt)) .setParallelism(1) - .name(FAILURE_MAPPER_NAME); + .name(FAILURE_MAPPER_NAME).uid("0006"); } eventStream3.keyBy(Event::getKey) .flatMap(createSemanticsCheckMapper(pt)) .name(SEMANTICS_CHECK_MAPPER_NAME) - .addSink(new PrintSinkFunction<>()); + .uid("0007") + .addSink(new PrintSinkFunction<>()).uid("0008"); env.execute("General purpose test job"); } From b7ba497d0d1b35839551d608debf4e989203205a Mon Sep 17 00:00:00 2001 From: azagrebin Date: Wed, 14 Nov 2018 10:08:39 +0100 Subject: [PATCH 068/359] [FLINK-10455][Kafka Tx] Close transactional producers in case of failure and termination (#6989) This commit addresses the problem of potential leak of resources associated with unclosed Kafka transactional producers in case of commitment failure or task shutdown. 1. always close producer even if commit fails in TwoPhaseCommitSinkFunction#notifyCheckpointComplete 2. close pending transactions in close method of Kafka Flink function in case of task shutdown 3. continue trying to commit other transactions in TwoPhaseCommitSinkFunction#notifyCheckpointComplete if any of them failed --- .../kafka/FlinkKafkaProducer011.java | 11 +++++++-- .../connectors/kafka/FlinkKafkaProducer.java | 11 +++++++-- .../sink/TwoPhaseCommitSinkFunction.java | 24 ++++++++++++++++++- 3 files changed, 41 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index c7f84c36b6ae8..3e7cf2b1aca58 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -45,6 +45,7 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.IOUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -671,6 +672,9 @@ public void close() throws FlinkKafka011Exception { } // make sure we propagate pending errors checkErroneous(); + pendingTransactions().forEach(transaction -> + IOUtils.closeQuietly(transaction.getValue().producer) + ); } // ------------------- Logic for handling checkpoint flushing -------------------------- // @@ -713,8 +717,11 @@ protected void preCommit(KafkaTransactionState transaction) throws FlinkKafka011 @Override protected void commit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { - transaction.producer.commitTransaction(); - recycleTransactionalProducer(transaction.producer); + try { + transaction.producer.commitTransaction(); + } finally { + recycleTransactionalProducer(transaction.producer); + } } } diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index df1a4b5727f91..10e8ef1713dce 100644 --- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -45,6 +45,7 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.IOUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -673,6 +674,9 @@ public void close() throws FlinkKafkaException { } // make sure we propagate pending errors checkErroneous(); + pendingTransactions().forEach(transaction -> + IOUtils.closeQuietly(transaction.getValue().producer) + ); } // ------------------- Logic for handling checkpoint flushing -------------------------- // @@ -715,8 +719,11 @@ protected void preCommit(FlinkKafkaProducer.KafkaTransactionState transaction) t @Override protected void commit(FlinkKafkaProducer.KafkaTransactionState transaction) { if (transaction.isTransactional()) { - transaction.producer.commitTransaction(); - recycleTransactionalProducer(transaction.producer); + try { + transaction.producer.commitTransaction(); + } finally { + recycleTransactionalProducer(transaction.producer); + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java index d2735d566eef7..e39335479f3c5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java @@ -39,20 +39,24 @@ import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; import java.time.Clock; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Stream; import static java.util.Objects.requireNonNull; import static org.apache.flink.util.Preconditions.checkArgument; @@ -150,6 +154,12 @@ protected TXN currentTransaction() { return currentTransactionHolder == null ? null : currentTransactionHolder.handle; } + @Nonnull + protected Stream> pendingTransactions() { + return pendingCommitTransactions.entrySet().stream() + .map(e -> new AbstractMap.SimpleEntry<>(e.getKey(), e.getValue().handle)); + } + // ------ methods that should be implemented in child class to support two phase commit algorithm ------ /** @@ -257,6 +267,7 @@ public final void notifyCheckpointComplete(long checkpointId) throws Exception { Iterator>> pendingTransactionIterator = pendingCommitTransactions.entrySet().iterator(); checkState(pendingTransactionIterator.hasNext(), "checkpoint completed, but no transaction pending"); + Throwable firstError = null; while (pendingTransactionIterator.hasNext()) { Map.Entry> entry = pendingTransactionIterator.next(); @@ -270,12 +281,23 @@ public final void notifyCheckpointComplete(long checkpointId) throws Exception { name(), checkpointId, pendingTransaction, pendingTransactionCheckpointId); logWarningIfTimeoutAlmostReached(pendingTransaction); - commit(pendingTransaction.handle); + try { + commit(pendingTransaction.handle); + } catch (Throwable t) { + if (firstError == null) { + firstError = t; + } + } LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction); pendingTransactionIterator.remove(); } + + if (firstError != null) { + throw new FlinkRuntimeException("Committing one of transactions failed, logging first encountered failure", + firstError); + } } @Override From 6c84103cbd2a7fa67e00e548b5df7161b6bd1ec7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 13 Nov 2018 15:04:11 +0100 Subject: [PATCH 069/359] [FLINK-10856] Take latest checkpoint to resume from in resume from externalized checkpoint e2e test Since it can happen that some empty checkpoint directories are left, we have to take the latest checkpoint directory in order to resume from an externalized checkpoint. This commit changes the test_resume_externalized_checkpoint.sh to sort the checkpoint directories in descending order and then takes the head checkpoint directory. --- .../test_resume_externalized_checkpoints.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh index fe2319063b54f..c1477574d5d7f 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh @@ -111,19 +111,14 @@ else cancel_job $DATASTREAM_JOB fi -CHECKPOINT_PATH=$(ls -d $CHECKPOINT_DIR/$DATASTREAM_JOB/chk-[1-9]*) +# take the latest checkpoint +CHECKPOINT_PATH=$(ls -d $CHECKPOINT_DIR/$DATASTREAM_JOB/chk-[1-9]* | sort -Vr | head -n1) if [ -z $CHECKPOINT_PATH ]; then echo "Expected an externalized checkpoint to be present, but none exists." exit 1 fi -NUM_CHECKPOINTS=$(echo $CHECKPOINT_PATH | wc -l | tr -d ' ') -if (( $NUM_CHECKPOINTS > 1 )); then - echo "Expected only exactly 1 externalized checkpoint to be present, but $NUM_CHECKPOINTS exists." - exit 1 -fi - echo "Restoring job with externalized checkpoint at $CHECKPOINT_PATH ..." BASE_JOB_CMD=`buildBaseJobCmd $NEW_DOP "-s file://${CHECKPOINT_PATH}"` @@ -141,6 +136,11 @@ fi DATASTREAM_JOB=$($JOB_CMD | grep "Job has been submitted with JobID" | sed 's/.* //g') +if [ -z $DATASTREAM_JOB ]; then + echo "Resuming from externalized checkpoint job could not be started." + exit 1 +fi + wait_job_running $DATASTREAM_JOB wait_oper_metric_num_in_records SemanticsCheckMapper.0 200 From 480d5e364ac905e36b8aa652c249d0d4c10eb219 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 12 Nov 2018 18:21:40 +0100 Subject: [PATCH 070/359] [FLINK-10419] Using DeclineCheckpoint message class when invoking RPC declineCheckpoint --- .../runtime/checkpoint/CheckpointCoordinatorGateway.java | 7 ++----- .../org/apache/flink/runtime/jobmaster/JobMaster.java | 8 +------- .../runtime/taskexecutor/rpc/RpcCheckpointResponder.java | 6 +++++- .../runtime/jobmaster/utils/TestingJobMasterGateway.java | 9 +++++---- .../jobmaster/utils/TestingJobMasterGatewayBuilder.java | 5 +++-- 5 files changed, 16 insertions(+), 19 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java index 22244f6cb8d51..b8dc554570617 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.rpc.RpcGateway; public interface CheckpointCoordinatorGateway extends RpcGateway { @@ -31,9 +32,5 @@ void acknowledgeCheckpoint( final CheckpointMetrics checkpointMetrics, final TaskStateSnapshot subtaskState); - void declineCheckpoint( - JobID jobID, - ExecutionAttemptID executionAttemptID, - long checkpointId, - Throwable cause); + void declineCheckpoint(DeclineCheckpoint declineCheckpoint); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 5d2d363cf71b6..40a675aca3146 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -690,13 +690,7 @@ public void acknowledgeCheckpoint( // TODO: This method needs a leader session ID @Override - public void declineCheckpoint( - final JobID jobID, - final ExecutionAttemptID executionAttemptID, - final long checkpointID, - final Throwable reason) { - final DeclineCheckpoint decline = new DeclineCheckpoint( - jobID, executionAttemptID, checkpointID, reason); + public void declineCheckpoint(DeclineCheckpoint decline) { final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); if (checkpointCoordinator != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java index c8f7357ab7e18..10c9e2f123ac1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.util.Preconditions; @@ -57,6 +58,9 @@ public void declineCheckpoint( long checkpointId, Throwable cause) { - checkpointCoordinatorGateway.declineCheckpoint(jobID, executionAttemptID, checkpointId, cause); + checkpointCoordinatorGateway.declineCheckpoint(new DeclineCheckpoint(jobID, + executionAttemptID, + checkpointId, + cause)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java index c9c55a1da8ff6..f5f7f8e3415c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobmaster.SerializedInputSplit; import org.apache.flink.runtime.jobmaster.message.ClassloadingProps; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.registration.RegistrationResponse; @@ -144,7 +145,7 @@ public class TestingJobMasterGateway implements JobMasterGateway { private final Consumer> acknowledgeCheckpointConsumer; @Nonnull - private final Consumer> declineCheckpointConsumer; + private final Consumer declineCheckpointConsumer; @Nonnull private final Supplier fencingTokenSupplier; @@ -183,7 +184,7 @@ public TestingJobMasterGateway( @Nonnull Function> requestOperatorBackPressureStatsFunction, @Nonnull BiConsumer notifyAllocationFailureConsumer, @Nonnull Consumer> acknowledgeCheckpointConsumer, - @Nonnull Consumer> declineCheckpointConsumer, + @Nonnull Consumer declineCheckpointConsumer, @Nonnull Supplier fencingTokenSupplier, @Nonnull BiFunction> requestKvStateLocationFunction, @Nonnull Function, CompletableFuture> notifyKvStateRegisteredFunction, @@ -335,8 +336,8 @@ public void acknowledgeCheckpoint(JobID jobID, ExecutionAttemptID executionAttem } @Override - public void declineCheckpoint(JobID jobID, ExecutionAttemptID executionAttemptID, long checkpointId, Throwable cause) { - declineCheckpointConsumer.accept(Tuple4.of(jobID, executionAttemptID, checkpointId, cause)); + public void declineCheckpoint(DeclineCheckpoint declineCheckpoint) { + declineCheckpointConsumer.accept(declineCheckpoint); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java index e40b752f24842..b52df9ee05233 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.jobmaster.SerializedInputSplit; import org.apache.flink.runtime.jobmaster.message.ClassloadingProps; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.UnknownKvStateLocation; @@ -96,7 +97,7 @@ public class TestingJobMasterGatewayBuilder { private Function> requestOperatorBackPressureStatsFunction = ignored -> CompletableFuture.completedFuture(OperatorBackPressureStatsResponse.of(null)); private BiConsumer notifyAllocationFailureConsumer = (ignoredA, ignoredB) -> {}; private Consumer> acknowledgeCheckpointConsumer = ignored -> {}; - private Consumer> declineCheckpointConsumer = ignored -> {}; + private Consumer declineCheckpointConsumer = ignored -> {}; private Supplier fencingTokenSupplier = () -> JOB_MASTER_ID; private BiFunction> requestKvStateLocationFunction = (ignoredA, registrationName) -> FutureUtils.completedExceptionally(new UnknownKvStateLocation(registrationName)); private Function, CompletableFuture> notifyKvStateRegisteredFunction = ignored -> CompletableFuture.completedFuture(Acknowledge.get()); @@ -222,7 +223,7 @@ public TestingJobMasterGatewayBuilder setAcknowledgeCheckpointConsumer(Consumer< return this; } - public TestingJobMasterGatewayBuilder setDeclineCheckpointConsumer(Consumer> declineCheckpointConsumer) { + public TestingJobMasterGatewayBuilder setDeclineCheckpointConsumer(Consumer declineCheckpointConsumer) { this.declineCheckpointConsumer = declineCheckpointConsumer; return this; } From d16e012a33adc23e2a77f68d2ef402ca931baa58 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 13 Nov 2018 11:42:08 +0100 Subject: [PATCH 071/359] [FLINK-10419] Added IT test to check declineCheckpoint invocation --- .../apache/flink/runtime/rpc/RpcUtils.java | 27 +++++++ .../runtime/jobmaster/JobMasterTest.java | 77 +++++++++++++++++++ 2 files changed, 104 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java index c90a8b5bbbc6e..2f656d092636b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java @@ -19,9 +19,13 @@ package org.apache.flink.runtime.rpc; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -87,6 +91,29 @@ public static void terminateRpcService(RpcService rpcService, Time timeout) thro rpcService.stopService().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } + /** + * Shuts the given rpc services down and waits for their termination. + * + * @param rpcServices to shut down + * @param timeout for this operation + * @throws InterruptedException if the operation has been interrupted + * @throws ExecutionException if a problem occurred + * @throws TimeoutException if a timeout occurred + */ + public static void terminateRpcServices( + Time timeout, + RpcService... rpcServices) throws InterruptedException, ExecutionException, TimeoutException { + final Collection> terminationFutures = new ArrayList<>(rpcServices.length); + + for (RpcService service : rpcServices) { + if (service != null) { + terminationFutures.add(service.stopService()); + } + } + + FutureUtils.waitForAll(terminationFutures).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + // We don't want this class to be instantiable private RpcUtils() {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index f3c72545e6950..b1f02845c1442 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -33,6 +33,7 @@ import org.apache.flink.core.io.InputSplitSource; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.queryablestate.KvStateID; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointProperties; @@ -84,14 +85,17 @@ import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.UnknownKvStateLocation; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.OperatorStreamStateHandle; @@ -106,12 +110,15 @@ import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.testutils.ClassLoaderUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.SupplierWithException; +import akka.actor.ActorSystem; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.junit.After; @@ -130,6 +137,7 @@ import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -248,6 +256,75 @@ public static void teardownClass() { } } + @Test + public void testDeclineCheckpointInvocationWithUserException() throws Exception { + RpcService rpcService1 = null; + RpcService rpcService2 = null; + try { + final ActorSystem actorSystem1 = AkkaUtils.createDefaultActorSystem(); + final ActorSystem actorSystem2 = AkkaUtils.createDefaultActorSystem(); + + rpcService1 = new AkkaRpcService(actorSystem1, testingTimeout); + rpcService2 = new AkkaRpcService(actorSystem2, testingTimeout); + + final CompletableFuture declineCheckpointMessageFuture = new CompletableFuture<>(); + + final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); + final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); + final JobMaster jobMaster = new JobMaster( + rpcService1, + jobMasterConfiguration, + jmResourceId, + jobGraph, + haServices, + DefaultSlotPoolFactory.fromConfiguration(configuration, rpcService1), + jobManagerSharedServices, + heartbeatServices, + blobServer, + UnregisteredJobManagerJobMetricGroupFactory.INSTANCE, + new NoOpOnCompletionActions(), + testingFatalErrorHandler, + JobMasterTest.class.getClassLoader()) { + @Override + public void declineCheckpoint(DeclineCheckpoint declineCheckpoint) { + declineCheckpointMessageFuture.complete(declineCheckpoint.getReason()); + } + }; + + jobMaster.start(jobMasterId, testingTimeout).get(); + + final String className = "UserException"; + final URLClassLoader userClassLoader = ClassLoaderUtils.compileAndLoadJava( + temporaryFolder.newFolder(), + className + ".java", + String.format("public class %s extends RuntimeException { public %s() {super(\"UserMessage\");} }", + className, + className)); + + Throwable userException = (Throwable) Class.forName(className, false, userClassLoader).newInstance(); + + CompletableFuture jobMasterGateway = + rpcService2.connect(jobMaster.getAddress(), jobMaster.getFencingToken(), JobMasterGateway.class); + + jobMasterGateway.thenAccept(gateway -> { + gateway.declineCheckpoint(new DeclineCheckpoint( + jobGraph.getJobID(), + new ExecutionAttemptID(1, 1), + 1, + userException + ) + ); + }); + + Throwable throwable = declineCheckpointMessageFuture.get(testingTimeout.toMilliseconds(), + TimeUnit.MILLISECONDS); + assertThat(throwable, instanceOf(SerializedThrowable.class)); + assertThat(throwable.getMessage(), equalTo(userException.getMessage())); + } finally { + RpcUtils.terminateRpcServices(testingTimeout, rpcService1, rpcService2); + } + } + @Test public void testHeartbeatTimeoutWithTaskManager() throws Exception { final CompletableFuture heartbeatResourceIdFuture = new CompletableFuture<>(); From ca28b071dd862fb88fd7f383cf8d7d144aca7cd7 Mon Sep 17 00:00:00 2001 From: azagrebin Date: Thu, 15 Nov 2018 10:09:59 +0100 Subject: [PATCH 072/359] [FLINK-10628][E2E][SSL] Enable mutual REST SSL auth in e2e tests --- docs/ops/security-ssl.md | 25 ++++- flink-end-to-end-tests/test-scripts/common.sh | 71 ++++---------- .../test-scripts/common_ssl.sh | 97 +++++++++++++++++++ .../test-scripts/test_batch_allround.sh | 2 +- .../test-scripts/test_streaming_file_sink.sh | 2 + 5 files changed, 139 insertions(+), 58 deletions(-) create mode 100644 flink-end-to-end-tests/test-scripts/common_ssl.sh diff --git a/docs/ops/security-ssl.md b/docs/ops/security-ssl.md index 4e3716218d206..77099ebf61f45 100644 --- a/docs/ops/security-ssl.md +++ b/docs/ops/security-ssl.md @@ -198,7 +198,7 @@ This example shows how to create a simple keystore / truststore pair. The trusts be shared with other applications. In this example, *myhost.company.org / ip:10.0.2.15* is the node (or service) for the Flink master. {% highlight bash %} -keytool -genkeypair -alias flink.rest -keystore rest.keystore -dname "CN=myhost.company.org" -ext "SAN=dns:myhost.company.org,ip:10.0.2.15" -storepass rest_keystore_password -keypass rest_key_password -keyalg RSA -keysize 4096 +keytool -genkeypair -alias flink.rest -keystore rest.keystore -dname "CN=myhost.company.org" -ext "SAN=dns:myhost.company.org,ip:10.0.2.15" -storepass rest_keystore_password -keypass rest_key_password -keyalg RSA -keysize 4096 -storetype PKCS12 keytool -exportcert -keystore rest.keystore -alias flink.rest -storepass rest_keystore_password -file flink.cer @@ -219,7 +219,7 @@ security.ssl.rest.key-password: rest_key_password Execute the following keytool commands to create a truststore with a self signed CA. {% highlight bash %} -keytool -genkeypair -alias ca -keystore ca.keystore -dname "CN=Sample CA" -storepass ca_keystore_password -keypass ca_key_password -keyalg RSA -keysize 4096 -ext "bc=ca:true" +keytool -genkeypair -alias ca -keystore ca.keystore -dname "CN=Sample CA" -storepass ca_keystore_password -keypass ca_key_password -keyalg RSA -keysize 4096 -ext "bc=ca:true" -storetype PKCS12 keytool -exportcert -keystore ca.keystore -alias ca -storepass ca_keystore_password -file ca.cer @@ -230,7 +230,7 @@ Now create a keystore for the REST endpoint with a certificate signed by the abo Let *flink.company.org / ip:10.0.2.15* be the hostname of the Flink master (JobManager). {% highlight bash %} -keytool -genkeypair -alias flink.rest -keystore rest.signed.keystore -dname "CN=flink.company.org" -ext "SAN=dns:flink.company.org" -storepass rest_keystore_password -keypass rest_key_password -keyalg RSA -keysize 4096 +keytool -genkeypair -alias flink.rest -keystore rest.signed.keystore -dname "CN=flink.company.org" -ext "SAN=dns:flink.company.org" -storepass rest_keystore_password -keypass rest_key_password -keyalg RSA -keysize 4096 -storetype PKCS12 keytool -certreq -alias flink.rest -keystore rest.signed.keystore -storepass rest_keystore_password -keypass rest_key_password -file rest.csr @@ -252,6 +252,25 @@ security.ssl.rest.key-password: rest_key_password security.ssl.rest.truststore-password: ca_truststore_password {% endhighlight %} +**Tips to query REST Endpoint with curl utility** + +You can convert the keystore into the `PEM` format using `openssl`: + +{% highlight bash %} +openssl pkcs12 -passin pass:rest_keystore_password -in rest.keystore -out rest.pem -nodes +{% endhighlight %} + +Then you can query REST Endpoint with `curl`: + +{% highlight bash %} +curl --cacert rest.pem flink_url +{% endhighlight %} + +If mutual SSL is enabled: + +{% highlight bash %} +curl --cacert rest.pem --cert rest.pem flink_url +{% endhighlight %} ## Tips for YARN / Mesos Deployment diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 0024d82f3d65a..ea2675386391c 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -41,13 +41,20 @@ echo "Flink dist directory: $FLINK_DIR" FLINK_VERSION=$(cat ${END_TO_END_DIR}/pom.xml | sed -n 's/.*\(.*\)<\/version>/\1/p') -USE_SSL=OFF # set via set_conf_ssl(), reset via revert_default_config() TEST_ROOT=`pwd -P` TEST_INFRA_DIR="$END_TO_END_DIR/test-scripts/" cd $TEST_INFRA_DIR TEST_INFRA_DIR=`pwd -P` cd $TEST_ROOT +NODENAME=`hostname -f` + +# REST_PROTOCOL and CURL_SSL_ARGS can be modified in common_ssl.sh if SSL is activated +# they should be used in curl command to query Flink REST API +REST_PROTOCOL="http" +CURL_SSL_ARGS="" +source "${TEST_INFRA_DIR}/common_ssl.sh" + function print_mem_use_osx { declare -a mem_types=("active" "inactive" "wired down") used="" @@ -87,7 +94,8 @@ function revert_default_config() { mv -f $FLINK_DIR/conf/flink-conf.yaml.bak $FLINK_DIR/conf/flink-conf.yaml fi - USE_SSL=OFF + REST_PROTOCOL="http" + CURL_SSL_ARGS="" } function set_conf() { @@ -170,46 +178,6 @@ function get_node_ip { echo ${ip_addr} } -function set_conf_ssl { - - # clean up the dir that will be used for SSL certificates and trust stores - if [ -e "${TEST_DATA_DIR}/ssl" ]; then - echo "File ${TEST_DATA_DIR}/ssl exists. Deleting it..." - rm -rf "${TEST_DATA_DIR}/ssl" - fi - mkdir -p "${TEST_DATA_DIR}/ssl" - - NODENAME=`hostname -f` - SANSTRING="dns:${NODENAME}" - for NODEIP in $(get_node_ip) ; do - SANSTRING="${SANSTRING},ip:${NODEIP}" - done - - echo "Using SAN ${SANSTRING}" - - # create certificates - keytool -genkeypair -alias ca -keystore "${TEST_DATA_DIR}/ssl/ca.keystore" -dname "CN=Sample CA" -storepass password -keypass password -keyalg RSA -ext bc=ca:true - keytool -keystore "${TEST_DATA_DIR}/ssl/ca.keystore" -storepass password -alias ca -exportcert > "${TEST_DATA_DIR}/ssl/ca.cer" - keytool -importcert -keystore "${TEST_DATA_DIR}/ssl/ca.truststore" -alias ca -storepass password -noprompt -file "${TEST_DATA_DIR}/ssl/ca.cer" - - keytool -genkeypair -alias node -keystore "${TEST_DATA_DIR}/ssl/node.keystore" -dname "CN=${NODENAME}" -ext SAN=${SANSTRING} -storepass password -keypass password -keyalg RSA - keytool -certreq -keystore "${TEST_DATA_DIR}/ssl/node.keystore" -storepass password -alias node -file "${TEST_DATA_DIR}/ssl/node.csr" - keytool -gencert -keystore "${TEST_DATA_DIR}/ssl/ca.keystore" -storepass password -alias ca -ext SAN=${SANSTRING} -infile "${TEST_DATA_DIR}/ssl/node.csr" -outfile "${TEST_DATA_DIR}/ssl/node.cer" - keytool -importcert -keystore "${TEST_DATA_DIR}/ssl/node.keystore" -storepass password -file "${TEST_DATA_DIR}/ssl/ca.cer" -alias ca -noprompt - keytool -importcert -keystore "${TEST_DATA_DIR}/ssl/node.keystore" -storepass password -file "${TEST_DATA_DIR}/ssl/node.cer" -alias node -noprompt - - # adapt config - # (here we rely on security.ssl.enabled enabling SSL for all components and internal as well as - # external communication channels) - set_conf security.ssl.enabled true - set_conf security.ssl.keystore ${TEST_DATA_DIR}/ssl/node.keystore - set_conf security.ssl.keystore-password password - set_conf security.ssl.key-password password - set_conf security.ssl.truststore ${TEST_DATA_DIR}/ssl/ca.truststore - set_conf security.ssl.truststore-password password - USE_SSL=ON -} - function start_ha_cluster { create_ha_config start_local_zk @@ -243,19 +211,13 @@ function start_local_zk { function wait_dispatcher_running { # wait at most 10 seconds until the dispatcher is up - local QUERY_URL - if [ "x$USE_SSL" = "xON" ]; then - QUERY_URL="http://localhost:8081/taskmanagers" - else - QUERY_URL="https://localhost:8081/taskmanagers" - fi + local QUERY_URL="${REST_PROTOCOL}://${NODENAME}:8081/taskmanagers" for i in {1..10}; do # without the || true this would exit our script if the JobManager is not yet up - QUERY_RESULT=$(curl "$QUERY_URL" 2> /dev/null || true) + QUERY_RESULT=$(curl ${CURL_SSL_ARGS} "$QUERY_URL" 2> /dev/null || true) # ensure the taskmanagers field is there at all and is not empty if [[ ${QUERY_RESULT} =~ \{\"taskmanagers\":\[.+\]\} ]]; then - echo "Dispatcher REST endpoint is up." break fi @@ -280,8 +242,9 @@ function start_taskmanagers { } function start_and_wait_for_tm { + local url="${REST_PROTOCOL}://${NODENAME}:8081/taskmanagers" - tm_query_result=$(curl -s "http://localhost:8081/taskmanagers") + tm_query_result=$(curl ${CURL_SSL_ARGS} -s "${url}") # we assume that the cluster is running if ! [[ ${tm_query_result} =~ \{\"taskmanagers\":\[.*\]\} ]]; then @@ -289,12 +252,12 @@ function start_and_wait_for_tm { exit 1 fi - running_tms=`curl -s "http://localhost:8081/taskmanagers" | grep -o "id" | wc -l` + running_tms=`curl ${CURL_SSL_ARGS} -s "${url}" | grep -o "id" | wc -l` ${FLINK_DIR}/bin/taskmanager.sh start for i in {1..10}; do - local new_running_tms=`curl -s "http://localhost:8081/taskmanagers" | grep -o "id" | wc -l` + local new_running_tms=`curl ${CURL_SSL_ARGS} -s "${url}" | grep -o "id" | wc -l` if [ $((new_running_tms-running_tms)) -eq 0 ]; then echo "TaskManager is not yet up." else @@ -541,7 +504,7 @@ function get_job_metric { local job_id=$1 local metric_name=$2 - local json=$(curl -s http://localhost:8081/jobs/${job_id}/metrics?get=${metric_name}) + local json=$(curl ${CURL_SSL_ARGS} -s ${REST_PROTOCOL}://${NODENAME}:8081/jobs/${job_id}/metrics?get=${metric_name}) local metric_value=$(echo ${json} | sed -n 's/.*"value":"\(.*\)".*/\1/p') echo ${metric_value} diff --git a/flink-end-to-end-tests/test-scripts/common_ssl.sh b/flink-end-to-end-tests/test-scripts/common_ssl.sh new file mode 100644 index 0000000000000..a5e19eaf835d0 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/common_ssl.sh @@ -0,0 +1,97 @@ +#!/usr/bin/env bash + +################################################################################ +# 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. +################################################################################ + +# NOTE: already sourced in common.sh + +function _set_conf_ssl_helper { + local type=$1 # 'internal' or external 'rest' + local ssl_dir="${TEST_DATA_DIR}/ssl/${type}" + local password="${type}.password" + + if [ "${type}" != "internal" ] && [ "${type}" != "rest" ]; then + echo "Unknown type of ssl connectivity: ${type}. It can be either 'internal' or external 'rest'" + exit 1 + fi + + # clean up the dir that will be used for SSL certificates and trust stores + if [ -e "${ssl_dir}" ]; then + echo "File ${ssl_dir} exists. Deleting it..." + rm -rf "${ssl_dir}" + fi + mkdir -p "${ssl_dir}" + + SANSTRING="dns:${NODENAME}" + for NODEIP in $(get_node_ip) ; do + SANSTRING="${SANSTRING},ip:${NODEIP}" + done + + echo "Using SAN ${SANSTRING}" + + # create certificates + keytool -genkeypair -alias ca -keystore "${ssl_dir}/ca.keystore" -dname "CN=Sample CA" -storepass ${password} -keypass ${password} -keyalg RSA -ext bc=ca:true -storetype PKCS12 + keytool -keystore "${ssl_dir}/ca.keystore" -storepass ${password} -alias ca -exportcert > "${ssl_dir}/ca.cer" + keytool -importcert -keystore "${ssl_dir}/ca.truststore" -alias ca -storepass ${password} -noprompt -file "${ssl_dir}/ca.cer" + + keytool -genkeypair -alias node -keystore "${ssl_dir}/node.keystore" -dname "CN=${NODENAME}" -ext SAN=${SANSTRING} -storepass ${password} -keypass ${password} -keyalg RSA -storetype PKCS12 + keytool -certreq -keystore "${ssl_dir}/node.keystore" -storepass ${password} -alias node -file "${ssl_dir}/node.csr" + keytool -gencert -keystore "${ssl_dir}/ca.keystore" -storepass ${password} -alias ca -ext SAN=${SANSTRING} -infile "${ssl_dir}/node.csr" -outfile "${ssl_dir}/node.cer" + keytool -importcert -keystore "${ssl_dir}/node.keystore" -storepass ${password} -file "${ssl_dir}/ca.cer" -alias ca -noprompt + keytool -importcert -keystore "${ssl_dir}/node.keystore" -storepass ${password} -file "${ssl_dir}/node.cer" -alias node -noprompt + + # keystore is converted into a pem format to use it as node.pem with curl in Flink REST API queries, see also $CURL_SSL_ARGS + openssl pkcs12 -passin pass:${password} -in "${ssl_dir}/node.keystore" -out "${ssl_dir}/node.pem" -nodes + + # adapt config + # (here we rely on security.ssl.enabled enabling SSL for all components and internal as well as + # external communication channels) + set_conf security.ssl.${type}.enabled true + set_conf security.ssl.${type}.keystore ${ssl_dir}/node.keystore + set_conf security.ssl.${type}.keystore-password ${password} + set_conf security.ssl.${type}.key-password ${password} + set_conf security.ssl.${type}.truststore ${ssl_dir}/ca.truststore + set_conf security.ssl.${type}.truststore-password ${password} +} + +function _set_conf_mutual_rest_ssl { + local auth="${1:-server}" # only 'server' or 'mutual' + local mutual="false" + local ssl_dir="${TEST_DATA_DIR}/ssl/rest" + if [ "${auth}" == "mutual" ]; then + CURL_SSL_ARGS="${CURL_SSL_ARGS} --cert ${ssl_dir}/node.pem" + mutual="true"; + fi + echo "Mutual ssl auth: ${mutual}" + set_conf security.ssl.rest.authentication-enabled ${mutual} +} + +function set_conf_rest_ssl { + local auth="${1:-server}" # only 'server' or 'mutual' + local ssl_dir="${TEST_DATA_DIR}/ssl/rest" + _set_conf_ssl_helper "rest" + _set_conf_mutual_rest_ssl ${auth} + REST_PROTOCOL="https" + CURL_SSL_ARGS="${CURL_SSL_ARGS} --cacert ${ssl_dir}/node.pem" +} + +function set_conf_ssl { + local auth="${1:-server}" # only 'server' or 'mutual' + _set_conf_ssl_helper "internal" + set_conf_rest_ssl ${auth} +} diff --git a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh index 2bdda8195d87d..502c12f7dbeaa 100755 --- a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh +++ b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh @@ -29,7 +29,7 @@ echo "Run DataSet-Allround-Test Program" echo "taskmanager.network.memory.min: 10485760" >> $FLINK_DIR/conf/flink-conf.yaml echo "taskmanager.network.memory.max: 10485760" >> $FLINK_DIR/conf/flink-conf.yaml -set_conf_ssl +set_conf_ssl "server" start_cluster start_taskmanagers 3 diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh index 50f5afc131256..6c8d0b8543566 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh @@ -22,6 +22,8 @@ OUT_TYPE="${1:-local}" # other type: s3 source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/common_s3.sh +set_conf_ssl "mutual" + OUT=out OUTPUT_PATH="$TEST_DATA_DIR/$OUT" S3_OUTPUT_PATH="s3://$ARTIFACTS_AWS_BUCKET/$OUT" From 48cf0e4f2727f8c569f905ab7c4b82b513c3ce10 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 14 Nov 2018 13:24:52 +0100 Subject: [PATCH 073/359] [FLINK-10419] Call JobMasterGateway through RpcCheckpointResponder in test --- .../runtime/jobmaster/JobMasterTest.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index b1f02845c1442..de3f787509853 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -103,6 +103,7 @@ import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskexecutor.rpc.RpcCheckpointResponder; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -303,18 +304,16 @@ public void declineCheckpoint(DeclineCheckpoint declineCheckpoint) { Throwable userException = (Throwable) Class.forName(className, false, userClassLoader).newInstance(); - CompletableFuture jobMasterGateway = - rpcService2.connect(jobMaster.getAddress(), jobMaster.getFencingToken(), JobMasterGateway.class); - - jobMasterGateway.thenAccept(gateway -> { - gateway.declineCheckpoint(new DeclineCheckpoint( - jobGraph.getJobID(), - new ExecutionAttemptID(1, 1), - 1, - userException - ) - ); - }); + JobMasterGateway jobMasterGateway = + rpcService2.connect(jobMaster.getAddress(), jobMaster.getFencingToken(), JobMasterGateway.class).get(); + + RpcCheckpointResponder rpcCheckpointResponder = new RpcCheckpointResponder(jobMasterGateway); + rpcCheckpointResponder.declineCheckpoint( + jobGraph.getJobID(), + new ExecutionAttemptID(1, 1), + 1, + userException + ); Throwable throwable = declineCheckpointMessageFuture.get(testingTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); From b6f7551127f6ec0a0022e5a49746502635b66287 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 14 Nov 2018 10:17:23 +0100 Subject: [PATCH 074/359] [FLINK-10767] Updated savepoint compatibility matrix with 1.7 --- docs/ops/upgrading.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/ops/upgrading.md b/docs/ops/upgrading.md index eb00a2b28c6ba..0451940723e6f 100644 --- a/docs/ops/upgrading.md +++ b/docs/ops/upgrading.md @@ -212,6 +212,7 @@ Savepoints are compatible across Flink versions as indicated by the table below: 1.4.x 1.5.x 1.6.x + 1.7.x Limitations @@ -224,6 +225,7 @@ Savepoints are compatible across Flink versions as indicated by the table below: + The maximum parallelism of a job that was migrated from Flink 1.1.x to 1.2.x+ is currently fixed as the parallelism of the job. This means that the parallelism can not be increased after migration. This limitation might be removed in a future bugfix release. @@ -236,6 +238,7 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O When migrating from Flink 1.2.x to Flink 1.3.x+, changing parallelism at the same time is not supported. Users have to first take a savepoint after migrating to Flink 1.3.x+, and then change parallelism. Savepoints created for CEP applications cannot be restored in 1.4.x+. @@ -248,6 +251,7 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O Migrating from Flink 1.3.0 to Flink 1.4.[0,1] will fail if the savepoint contains Scala case classes. Users have to directly migrate to 1.4.2+ instead. @@ -258,6 +262,7 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O @@ -268,6 +273,7 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O + O @@ -278,6 +284,18 @@ Savepoints are compatible across Flink versions as indicated by the table below: O + O + + + + 1.7.x + + + + + + + O From 62f9fbbdaa00650dec21da7f4b32c3e383ac0a8d Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 13 Nov 2018 22:00:01 +0800 Subject: [PATCH 075/359] [FLINK-10624] Extend SQL client end-to-end to test new KafkaTableSink This closes #6927. --- .../flink-sql-client-test/pom.xml | 17 +- flink-end-to-end-tests/run-nightly-tests.sh | 2 + .../test-scripts/kafka_sql_common.sh | 101 ++++++++ .../test-scripts/test_sql_client.sh | 229 +++--------------- .../test-scripts/test_sql_client_kafka.sh | 22 ++ .../test-scripts/test_sql_client_kafka010.sh | 22 ++ .../test_sql_client_kafka_common.sh | 220 +++++++++++++++++ 7 files changed, 413 insertions(+), 200 deletions(-) create mode 100644 flink-end-to-end-tests/test-scripts/kafka_sql_common.sh create mode 100755 flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh create mode 100755 flink-end-to-end-tests/test-scripts/test_sql_client_kafka010.sh create mode 100755 flink-end-to-end-tests/test-scripts/test_sql_client_kafka_common.sh diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml index de1db9e6bf596..7710d01dda9bd 100644 --- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml +++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml @@ -89,6 +89,14 @@ under the License. sql-jar provided + + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + ${project.version} + sql-jar + provided + org.apache.flink @@ -106,7 +114,7 @@ under the License. as we neither access nor package the kafka dependencies --> org.apache.kafka kafka-clients - 0.11.0.2 + 2.0.0 @@ -176,6 +184,13 @@ under the License. sql-jar jar + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + ${project.version} + sql-jar + jar + + + + flink-end-to-end-tests + org.apache.flink + 1.8-SNAPSHOT + + + 4.0.0 + + flink-metrics-availability-test + 1.8-SNAPSHOT + + + + org.apache.flink + flink-core + ${project.version} + test + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${project.version} + test + + + org.apache.flink + flink-end-to-end-tests-common + ${project.version} + test + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + + + + + e2e-metric-availability + + + e2e-metrics + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + e2e-metric-availability + integration-test + + test + + + + **/*ITCase.* + + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java new file mode 100644 index 0000000000000..8b0cd3310969d --- /dev/null +++ b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java @@ -0,0 +1,161 @@ +/* + * 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.pache.flink.metrics.tests; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.RestClientConfiguration; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.JobManagerMetricsMessageParameters; +import org.apache.flink.runtime.rest.messages.job.metrics.MetricCollectionResponseBody; +import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsMessageParameters; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; +import org.apache.flink.tests.util.FlinkDistribution; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.SupplierWithException; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * End-to-end test for the availability of metrics. + */ +public class MetricsAvailabilityITCase extends TestLogger { + + private static final String HOST = "localhost"; + private static final int PORT = 8081; + + @Rule + public final FlinkDistribution dist = new FlinkDistribution(); + + @Nullable + private static ScheduledExecutorService scheduledExecutorService = null; + + @BeforeClass + public static void startExecutor() { + scheduledExecutorService = Executors.newScheduledThreadPool(4); + } + + @AfterClass + public static void shutdownExecutor() { + if (scheduledExecutorService != null) { + scheduledExecutorService.shutdown(); + } + } + + @Test + public void testReporter() throws Exception { + dist.startFlinkCluster(); + + final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), scheduledExecutorService); + + checkJobManagerMetricAvailability(restClient); + + final Collection taskManagerIds = getTaskManagerIds(restClient); + + for (final ResourceID taskManagerId : taskManagerIds) { + checkTaskManagerMetricAvailability(restClient, taskManagerId); + } + } + + private static void checkJobManagerMetricAvailability(final RestClient restClient) throws Exception { + final JobManagerMetricsHeaders headers = JobManagerMetricsHeaders.getInstance(); + final JobManagerMetricsMessageParameters parameters = headers.getUnresolvedMessageParameters(); + parameters.metricsFilterParameter.resolve(Collections.singletonList("numRegisteredTaskManagers")); + + fetchMetric(() -> + restClient.sendRequest(HOST, PORT, headers, parameters, EmptyRequestBody.getInstance()), + getMetricNamePredicate("numRegisteredTaskManagers")); + } + + private static Collection getTaskManagerIds(final RestClient restClient) throws Exception { + final TaskManagersHeaders headers = TaskManagersHeaders.getInstance(); + + final TaskManagersInfo response = fetchMetric(() -> + restClient.sendRequest( + HOST, + PORT, + headers, + EmptyMessageParameters.getInstance(), + EmptyRequestBody.getInstance()), + taskManagersInfo -> !taskManagersInfo.getTaskManagerInfos().isEmpty()); + + return response.getTaskManagerInfos().stream() + .map(TaskManagerInfo::getResourceId) + .collect(Collectors.toList()); + } + + private static void checkTaskManagerMetricAvailability(final RestClient restClient, final ResourceID taskManagerId) throws Exception { + final TaskManagerMetricsHeaders headers = TaskManagerMetricsHeaders.getInstance(); + final TaskManagerMetricsMessageParameters parameters = headers.getUnresolvedMessageParameters(); + parameters.taskManagerIdParameter.resolve(taskManagerId); + parameters.metricsFilterParameter.resolve(Collections.singletonList("Status.Network.TotalMemorySegments")); + + fetchMetric(() -> + restClient.sendRequest(HOST, PORT, headers, parameters, EmptyRequestBody.getInstance()), + getMetricNamePredicate("Status.Network.TotalMemorySegments")); + } + + private static X fetchMetric(final SupplierWithException, IOException> clientOperation, final Predicate predicate) throws InterruptedException, ExecutionException, TimeoutException { + final CompletableFuture responseFuture = FutureUtils.retrySuccesfulWithDelay(() -> { + try { + return clientOperation.get(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }, + Time.seconds(1), + Deadline.fromNow(Duration.ofSeconds(5)), + predicate, + new ScheduledExecutorServiceAdapter(scheduledExecutorService)); + + return responseFuture.get(30, TimeUnit.SECONDS); + } + + private static Predicate getMetricNamePredicate(final String metricName) { + return response -> response.getMetrics().stream().anyMatch(metric -> metric.getId().equals(metricName)); + } +} diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/resources/log4j-test.properties b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..f7425cd14c098 --- /dev/null +++ b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/resources/log4j-test.properties @@ -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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +log4j.rootLogger=INFO, testlogger + +# A1 is set to be a ConsoleAppender. +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%m%n diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index c6644e25a9460..90c32832af1cc 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -58,6 +58,7 @@ under the License. flink-state-evolution-test flink-e2e-test-utils flink-end-to-end-tests-common + flink-metrics-availability-test flink-metrics-reporter-prometheus-test flink-heavy-deployment-stress-test diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsMessageParameters.java index 7fd63726e0e21..311a30034c80c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsMessageParameters.java @@ -31,7 +31,7 @@ */ public class JobManagerMetricsMessageParameters extends MessageParameters { - private final MetricsFilterParameter metricsFilterParameter = new MetricsFilterParameter(); + public final MetricsFilterParameter metricsFilterParameter = new MetricsFilterParameter(); @Override public Collection> getPathParameters() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsMessageParameters.java index d7e9381d18525..b30e4b8b3ec5d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsMessageParameters.java @@ -31,7 +31,7 @@ */ public class TaskManagerMetricsMessageParameters extends TaskManagerMessageParameters { - private final MetricsFilterParameter metricsFilterParameter = new MetricsFilterParameter(); + public final MetricsFilterParameter metricsFilterParameter = new MetricsFilterParameter(); @Override public Collection> getQueryParameters() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java index 59408a7304e5e..4a761215bc102 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java @@ -31,7 +31,7 @@ */ public class TaskManagerMessageParameters extends MessageParameters { - private TaskManagerIdPathParameter taskManagerIdParameter = new TaskManagerIdPathParameter(); + public TaskManagerIdPathParameter taskManagerIdParameter = new TaskManagerIdPathParameter(); @Override public Collection> getPathParameters() { From d389e5a42e3d08f33f5f6c73f389fe8d81ae5bc3 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 14 Nov 2018 11:40:35 +0100 Subject: [PATCH 078/359] [hotfix][tests] Fix typo in FutureUtils#retrySuccessfulWithDelay --- .../flink/metrics/tests/MetricsAvailabilityITCase.java | 2 +- .../flink/runtime/jobmanager/JMXJobManagerMetricTest.java | 2 +- .../itcases/AbstractQueryableStateTestBase.java | 2 +- .../org/apache/flink/runtime/concurrent/FutureUtils.java | 2 +- .../taskmanager/TaskCancelAsyncProducerConsumerITCase.java | 4 ++-- .../flink/test/accumulators/AccumulatorLiveITCase.java | 2 +- .../apache/flink/test/checkpointing/SavepointITCase.java | 2 +- .../test/checkpointing/ZooKeeperHighAvailabilityITCase.java | 2 +- .../recovery/JobManagerHAProcessFailureRecoveryITCase.java | 2 +- .../flink/test/recovery/ProcessFailureCancelingITCase.java | 4 ++-- .../operator/restore/AbstractOperatorRestoreTestBase.java | 6 +++--- 11 files changed, 15 insertions(+), 15 deletions(-) diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java index 8b0cd3310969d..28a21354e5d27 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java +++ b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java @@ -140,7 +140,7 @@ private static void checkTaskManagerMetricAvailability(final RestClient restClie } private static X fetchMetric(final SupplierWithException, IOException> clientOperation, final Predicate predicate) throws InterruptedException, ExecutionException, TimeoutException { - final CompletableFuture responseFuture = FutureUtils.retrySuccesfulWithDelay(() -> { + final CompletableFuture responseFuture = FutureUtils.retrySuccessfulWithDelay(() -> { try { return clientOperation.get(); } catch (IOException e) { diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java index 915c02b644d8f..036796c582d45 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java @@ -107,7 +107,7 @@ public void testJobManagerJMXMetricAccess() throws Exception { client.setDetached(true); client.submitJob(jobGraph, JMXJobManagerMetricTest.class.getClassLoader()); - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> client.getJobStatus(jobGraph.getJobID()), Time.milliseconds(10), deadline, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 83976f16094a4..eb6b3ef8c50e8 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -1273,7 +1273,7 @@ public void close() throws Exception { // Free cluster resources clusterClient.cancel(jobId); // cancel() is non-blocking so do this to make sure the job finished - CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + CompletableFuture jobStatusFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobId), Time.milliseconds(50), deadline, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index 7392030496417..0f36a3af141db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -239,7 +239,7 @@ private static void retryOperationWithDelay( * @return Future which retries the given operation a given amount of times and delays the retry * in case the predicate isn't matched */ - public static CompletableFuture retrySuccesfulWithDelay( + public static CompletableFuture retrySuccessfulWithDelay( final Supplier> operation, final Time retryDelay, final Deadline deadline, 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 14aecc700ad2b..ddae8462539ac 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 @@ -109,7 +109,7 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { // Submit job and wait until running flink.runDetached(jobGraph); - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> flink.getJobStatus(jobGraph.getJobID()), Time.milliseconds(10), deadline, @@ -160,7 +160,7 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); // wait until the job is canceled - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> flink.getJobStatus(jobGraph.getJobID()), Time.milliseconds(10), deadline, diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 0ead861adbaf7..d4063184b62d4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -153,7 +153,7 @@ public void go() throws Exception { try { NotifyingMapper.notifyLatch.await(); - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> { try { return CompletableFuture.completedFuture(client.getAccumulators(jobGraph.getJobID())); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 2cd2bbb60e926..4f607691c1bec 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -200,7 +200,7 @@ private void restoreJobAndVerifyState(String savepointPath, MiniClusterResourceF client.cancel(jobId); - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> client.getJobStatus(jobId), Time.milliseconds(50), Deadline.now().plus(Duration.ofSeconds(30)), diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java index 7c00de7f43684..9bec331608ea6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -258,7 +258,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO // now the job should be able to go to RUNNING again and then eventually to FINISHED, // which it only does if it could successfully restore - CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + CompletableFuture jobStatusFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobID), Time.milliseconds(50), deadline, diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index ec3f1e1b9d3fa..e83f9ab8c583f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -393,7 +393,7 @@ public void run() { } private void waitForTaskManagers(int numberOfTaskManagers, DispatcherGateway dispatcherGateway, FiniteDuration timeLeft) throws ExecutionException, InterruptedException { - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> dispatcherGateway.requestClusterOverview(Time.milliseconds(timeLeft.toMillis())), Time.milliseconds(50L), org.apache.flink.api.common.time.Deadline.fromNow(Duration.ofMillis(timeLeft.toMillis())), diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index 90ea8796ca57c..ed987d677d8bb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -294,7 +294,7 @@ static DispatcherGateway retrieveDispatcherGateway(RpcService rpcService, HighAv } private void waitUntilAllSlotsAreUsed(DispatcherGateway dispatcherGateway, Time timeout) throws ExecutionException, InterruptedException { - FutureUtils.retrySuccesfulWithDelay( + FutureUtils.retrySuccessfulWithDelay( () -> dispatcherGateway.requestClusterOverview(timeout), Time.milliseconds(50L), Deadline.fromNow(Duration.ofMillis(timeout.toMilliseconds())), @@ -306,7 +306,7 @@ private void waitUntilAllSlotsAreUsed(DispatcherGateway dispatcherGateway, Time } private Collection waitForRunningJobs(ClusterClient clusterClient, Time timeout) throws ExecutionException, InterruptedException { - return FutureUtils.retrySuccesfulWithDelay( + return FutureUtils.retrySuccessfulWithDelay( CheckedSupplier.unchecked(clusterClient::listJobs), Time.milliseconds(50L), Deadline.fromNow(Duration.ofMillis(timeout.toMilliseconds())), diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index 097616feb9663..0b79af513f6bb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -118,7 +118,7 @@ private String migrateJob(ClassLoader classLoader, ClusterClient clusterClien clusterClient.submitJob(jobToMigrate, classLoader); - CompletableFuture jobRunningFuture = FutureUtils.retrySuccesfulWithDelay( + CompletableFuture jobRunningFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobToMigrate.getJobID()), Time.milliseconds(50), deadline, @@ -152,7 +152,7 @@ private String migrateJob(ClassLoader classLoader, ClusterClient clusterClien assertNotNull("Could not take savepoint.", savepointPath); - CompletableFuture jobCanceledFuture = FutureUtils.retrySuccesfulWithDelay( + CompletableFuture jobCanceledFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobToMigrate.getJobID()), Time.milliseconds(50), deadline, @@ -173,7 +173,7 @@ private void restoreJob(ClassLoader classLoader, ClusterClient clusterClient, clusterClient.submitJob(jobToRestore, classLoader); - CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + CompletableFuture jobStatusFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobToRestore.getJobID()), Time.milliseconds(50), deadline, From 16925c20dde883de37f1de65c7a9224d51a6254f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 15 Nov 2018 13:05:07 +0100 Subject: [PATCH 079/359] [FLINK-10857][metrics] Cache logical scopes separately for each reporter --- .../metrics/groups/AbstractMetricGroup.java | 37 +++++++++--- .../metrics/groups/FrontMetricGroup.java | 2 +- .../groups/AbstractMetricGroupTest.java | 56 +++++++++++++++++++ 3 files changed, 86 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java index 909915f216e99..4400b14a10d69 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java @@ -90,9 +90,9 @@ public abstract class AbstractMetricGroup> impl * For example: "host-7.taskmanager-2.window_word_count.my-mapper" */ private final String[] scopeStrings; - /** The logical metrics scope represented by this group, as a concatenated string, lazily computed. + /** The logical metrics scope represented by this group for each reporter, as a concatenated string, lazily computed. * For example: "taskmanager.job.task" */ - private String logicalScopeString; + private String[] logicalScopeStrings; /** The metrics query service scope represented by this group, lazily computed. */ protected QueryScopeInfo queryServiceScopeInfo; @@ -107,6 +107,7 @@ public AbstractMetricGroup(MetricRegistry registry, String[] scope, A parent) { this.scopeComponents = checkNotNull(scope); this.parent = parent; this.scopeStrings = new String[registry.getNumberReporters()]; + this.logicalScopeStrings = new String[registry.getNumberReporters()]; } public Map getAllVariables() { @@ -152,14 +153,34 @@ public String getLogicalScope(CharacterFilter filter) { * @return logical scope */ public String getLogicalScope(CharacterFilter filter, char delimiter) { - if (logicalScopeString == null) { - if (parent == null) { - logicalScopeString = getGroupName(filter); - } else { - logicalScopeString = parent.getLogicalScope(filter, delimiter) + delimiter + getGroupName(filter); + return getLogicalScope(filter, delimiter, -1); + } + + /** + * Returns the logical scope of this group, for example + * {@code "taskmanager.job.task"}. + * + * @param filter character filter which is applied to the scope components + * @param delimiter delimiter to use for concatenating scope components + * @param reporterIndex index of the reporter + * @return logical scope + */ + String getLogicalScope(CharacterFilter filter, char delimiter, int reporterIndex) { + if (logicalScopeStrings.length == 0 || (reporterIndex < 0 || reporterIndex >= logicalScopeStrings.length)) { + return createLogicalScope(filter, delimiter); + } else { + if (logicalScopeStrings[reporterIndex] == null) { + logicalScopeStrings[reporterIndex] = createLogicalScope(filter, delimiter); } + return logicalScopeStrings[reporterIndex]; } - return logicalScopeString; + } + + private String createLogicalScope(CharacterFilter filter, char delimiter) { + final String groupName = getGroupName(filter); + return parent == null + ? groupName + : parent.getLogicalScope(filter, delimiter) + delimiter + groupName; } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/FrontMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/FrontMetricGroup.java index 63842fef9d66e..64397d33db6bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/FrontMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/FrontMetricGroup.java @@ -52,6 +52,6 @@ public String getLogicalScope(CharacterFilter filter) { } public String getLogicalScope(CharacterFilter filter, char delimiter) { - return parentMetricGroup.getLogicalScope(filter, delimiter); + return parentMetricGroup.getLogicalScope(filter, delimiter, this.reporterIndex); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java index f8ed3c6a6d8e0..f3f8b42b85154 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroupTest.java @@ -105,6 +105,30 @@ public void testScopeCachingForMultipleReporters() throws Exception { } } + @Test + public void testLogicalScopeCachingForMultipleReporters() throws Exception { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, LogicalScopeReporter1.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, LogicalScopeReporter2.class.getName()); + + MetricRegistryImpl testRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); + try { + MetricGroup tmGroup = new TaskManagerMetricGroup(testRegistry, "host", "id") + .addGroup("B") + .addGroup("C"); + tmGroup.counter("1"); + assertEquals("Reporters were not properly instantiated", 2, testRegistry.getReporters().size()); + for (MetricReporter reporter : testRegistry.getReporters()) { + ScopeCheckingTestReporter typedReporter = (ScopeCheckingTestReporter) reporter; + if (typedReporter.failureCause != null) { + throw typedReporter.failureCause; + } + } + } finally { + testRegistry.shutdown().get(); + } + } + private abstract static class ScopeCheckingTestReporter extends TestReporter { protected Exception failureCause; @@ -175,6 +199,38 @@ public String filterCharacters(String input) { } } + /** + * Reporter that verifies the logical-scope caching behavior. + */ + public static final class LogicalScopeReporter1 extends ScopeCheckingTestReporter { + @Override + public String filterCharacters(String input) { + return FILTER_B.filterCharacters(input); + } + + @Override + public void checkScopes(Metric metric, String metricName, MetricGroup group) { + final String logicalScope = ((FrontMetricGroup>) group).getLogicalScope(this, '-'); + assertEquals("taskmanager-X-C", logicalScope); + } + } + + /** + * Reporter that verifies the logical-scope caching behavior. + */ + public static final class LogicalScopeReporter2 extends ScopeCheckingTestReporter { + @Override + public String filterCharacters(String input) { + return FILTER_C.filterCharacters(input); + } + + @Override + public void checkScopes(Metric metric, String metricName, MetricGroup group) { + final String logicalScope = ((FrontMetricGroup>) group).getLogicalScope(this, ','); + assertEquals("taskmanager,B,X", logicalScope); + } + } + @Test public void testScopeGenerationWithoutReporters() throws Exception { Configuration config = new Configuration(); From 60e9dcb62b62acd35496c0de27f38c1658768100 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 13:43:44 +0100 Subject: [PATCH 080/359] [FLINK-10899] Remove explicit version tag from flink-metrics-availability-test and flink-metrics-reporter-prometheus-test --- .../flink-metrics-availability-test/pom.xml | 4 ++-- .../flink-metrics-reporter-prometheus-test/pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml index eea4ef8458daa..9027498c5523a 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml @@ -22,13 +22,13 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT + .. 4.0.0 flink-metrics-availability-test - 1.8-SNAPSHOT diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml index 844c9b09346a8..4163d6a69eacf 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml @@ -23,12 +23,12 @@ under the License. flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT + .. 4.0.0 flink-metrics-reporter-prometheus-test - 1.7-SNAPSHOT From 4597daf881f24547f3866db7a9a20a606065115b Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 15 Nov 2018 11:59:24 +0100 Subject: [PATCH 081/359] [FLINK-10764][tests] Add ITCase for checkpoint path entropy injection. (#7075) Add a test that verifies that checkpoint data on the file system has additional entropy added to its path. Remove code duplication in SavepointITCase. --- .../EntropyInjectingTestFileSystem.java | 65 ++++++ ...org.apache.flink.core.fs.FileSystemFactory | 3 +- .../test/checkpointing/SavepointITCase.java | 200 ++++++++++++------ 3 files changed, 205 insertions(+), 63 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/testutils/EntropyInjectingTestFileSystem.java diff --git a/flink-core/src/test/java/org/apache/flink/testutils/EntropyInjectingTestFileSystem.java b/flink-core/src/test/java/org/apache/flink/testutils/EntropyInjectingTestFileSystem.java new file mode 100644 index 0000000000000..ef1575bea7446 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/testutils/EntropyInjectingTestFileSystem.java @@ -0,0 +1,65 @@ +/* + * 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.testutils; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.EntropyInjectingFileSystem; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemFactory; +import org.apache.flink.core.fs.local.LocalFileSystem; + +import java.net.URI; + +/** + * A test file system that implements {@link EntropyInjectingFileSystem}. + */ +public class EntropyInjectingTestFileSystem extends LocalFileSystem implements EntropyInjectingFileSystem { + + public static final String ENTROPY_INJECTION_KEY = "_entropy_"; + + public static final String ENTROPY = "_resolved_"; + + @Override + public String getEntropyInjectionKey() { + return ENTROPY_INJECTION_KEY; + } + + @Override + public String generateEntropy() { + return ENTROPY; + } + + public static class EntropyInjectingTestFileSystemFactory implements FileSystemFactory { + + @Override + public String getScheme() { + return "test-entropy"; + } + + @Override + public void configure(final Configuration config) { + } + + @Override + public FileSystem create(final URI fsUri) { + return new EntropyInjectingTestFileSystem(); + } + } +} diff --git a/flink-core/src/test/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory b/flink-core/src/test/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory index 5a3a31de23566..76965cb214543 100644 --- a/flink-core/src/test/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory +++ b/flink-core/src/test/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.testutils.TestFileSystem$TestFileSystemFactory \ No newline at end of file +org.apache.flink.testutils.TestFileSystem$TestFileSystemFactory +org.apache.flink.testutils.EntropyInjectingTestFileSystem$EntropyInjectingTestFileSystemFactory diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 4f607691c1bec..9140570a89009 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -53,11 +53,16 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.EntropyInjectingTestFileSystem; import org.apache.flink.util.Collector; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeDiagnosingMatcher; import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -66,7 +71,12 @@ import java.io.File; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; +import java.nio.file.FileVisitOption; +import java.nio.file.Files; +import java.nio.file.Path; import java.time.Duration; import java.util.Arrays; import java.util.Collections; @@ -76,9 +86,12 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -93,6 +106,22 @@ public class SavepointITCase extends TestLogger { @Rule public final TemporaryFolder folder = new TemporaryFolder(); + private File checkpointDir; + + private File savepointDir; + + @Before + public void setUp() throws Exception { + final File testRoot = folder.newFolder(); + + checkpointDir = new File(testRoot, "checkpoints"); + savepointDir = new File(testRoot, "savepoints"); + + if (!checkpointDir.mkdir() || !savepointDir.mkdirs()) { + fail("Test setup failed: failed to create temporary directories."); + } + } + /** * Triggers a savepoint for a job that uses the FsStateBackend. We expect * that all checkpoint files are written to a new savepoint directory. @@ -109,35 +138,46 @@ public class SavepointITCase extends TestLogger { */ @Test public void testTriggerSavepointAndResumeWithFileBasedCheckpoints() throws Exception { - // Config final int numTaskManagers = 2; final int numSlotsPerTaskManager = 2; final int parallelism = numTaskManagers * numSlotsPerTaskManager; - final File testRoot = folder.newFolder(); - Configuration config = new Configuration(); + final MiniClusterResourceFactory clusterFactory = new MiniClusterResourceFactory( + numTaskManagers, + numSlotsPerTaskManager, + getFileBasedCheckpointsConfig()); - final File checkpointDir = new File(testRoot, "checkpoints"); - final File savepointRootDir = new File(testRoot, "savepoints"); + final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism); + verifySavepoint(parallelism, savepointPath); - if (!checkpointDir.mkdir() || !savepointRootDir.mkdirs()) { - fail("Test setup failed: failed to create temporary directories."); - } + restoreJobAndVerifyState(savepointPath, clusterFactory, parallelism); + } - // Use file based checkpoints - config.setString(CheckpointingOptions.STATE_BACKEND, "filesystem"); - config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); - config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0); - config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointRootDir.toURI().toString()); + @Test + public void testShouldAddEntropyToSavepointPath() throws Exception { + final int numTaskManagers = 2; + final int numSlotsPerTaskManager = 2; + final int parallelism = numTaskManagers * numSlotsPerTaskManager; - MiniClusterResourceFactory clusterFactory = new MiniClusterResourceFactory(numTaskManagers, numSlotsPerTaskManager, config); + final MiniClusterResourceFactory clusterFactory = new MiniClusterResourceFactory( + numTaskManagers, + numSlotsPerTaskManager, + getCheckpointingWithEntropyConfig()); - String savepointPath = submitJobAndGetVerifiedSavepoint(clusterFactory, parallelism); + final String savepointPath = submitJobAndTakeSavepoint(clusterFactory, parallelism); + assertThat(savepointDir, hasEntropyInFileStateHandlePaths()); restoreJobAndVerifyState(savepointPath, clusterFactory, parallelism); } - private String submitJobAndGetVerifiedSavepoint(MiniClusterResourceFactory clusterFactory, int parallelism) throws Exception { + private Configuration getCheckpointingWithEntropyConfig() { + final String savepointPathWithEntropyPlaceholder = new File(savepointDir, EntropyInjectingTestFileSystem.ENTROPY_INJECTION_KEY).getPath(); + final Configuration config = getFileBasedCheckpointsConfig("test-entropy://" + savepointPathWithEntropyPlaceholder); + config.setString("s3.entropy.key", EntropyInjectingTestFileSystem.ENTROPY_INJECTION_KEY); + return config; + } + + private String submitJobAndTakeSavepoint(MiniClusterResourceFactory clusterFactory, int parallelism) throws Exception { final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000); final JobID jobId = jobGraph.getJobID(); StatefulCounter.resetForTest(parallelism); @@ -152,32 +192,32 @@ private String submitJobAndGetVerifiedSavepoint(MiniClusterResourceFactory clust StatefulCounter.getProgressLatch().await(); - String savepointPath = client.triggerSavepoint(jobId, null).get(); - - // Only one savepoint should exist - File savepointDir = new File(new URI(savepointPath)); - assertTrue("Savepoint directory does not exist.", savepointDir.exists()); - assertTrue("Savepoint did not create self-contained directory.", savepointDir.isDirectory()); - - File[] savepointFiles = savepointDir.listFiles(); - - if (savepointFiles != null) { - // Expect one metadata file and one checkpoint file per stateful - // parallel subtask - String errMsg = "Did not write expected number of savepoint/checkpoint files to directory: " - + Arrays.toString(savepointFiles); - assertEquals(errMsg, 1 + parallelism, savepointFiles.length); - } else { - fail(String.format("Returned savepoint path (%s) is not valid.", savepointPath)); - } - - return savepointPath; + return client.triggerSavepoint(jobId, null).get(); } finally { cluster.after(); StatefulCounter.resetForTest(parallelism); } } + private void verifySavepoint(final int parallelism, final String savepointPath) throws URISyntaxException { + // Only one savepoint should exist + File savepointDir = new File(new URI(savepointPath)); + assertTrue("Savepoint directory does not exist.", savepointDir.exists()); + assertTrue("Savepoint did not create self-contained directory.", savepointDir.isDirectory()); + + File[] savepointFiles = savepointDir.listFiles(); + + if (savepointFiles != null) { + // Expect one metadata file and one checkpoint file per stateful + // parallel subtask + String errMsg = "Did not write expected number of savepoint/checkpoint files to directory: " + + Arrays.toString(savepointFiles); + assertEquals(errMsg, 1 + parallelism, savepointFiles.length); + } else { + fail(String.format("Returned savepoint path (%s) is not valid.", savepointPath)); + } + } + private void restoreJobAndVerifyState(String savepointPath, MiniClusterResourceFactory clusterFactory, int parallelism) throws Exception { final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000); jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); @@ -224,9 +264,6 @@ public void testTriggerSavepointForNonExistingJob() throws Exception { final int numTaskManagers = 1; final int numSlotsPerTaskManager = 1; - final File tmpDir = folder.newFolder(); - final File savepointDir = new File(tmpDir, "savepoints"); - final Configuration config = new Configuration(); config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); @@ -299,9 +336,6 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { int numSlotsPerTaskManager = 1; int parallelism = numTaskManagers * numSlotsPerTaskManager; - final File tmpDir = folder.newFolder(); - final File savepointDir = new File(tmpDir, "savepoints"); - final Configuration config = new Configuration(); config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); @@ -361,9 +395,6 @@ public void testCanRestoreWithModifiedStatelessOperators() throws Exception { // Test deadline final Deadline deadline = Deadline.now().plus(Duration.ofMinutes(5)); - final File tmpDir = folder.newFolder(); - final File savepointDir = new File(tmpDir, "savepoints"); - // Flink configuration final Configuration config = new Configuration(); config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); @@ -590,11 +621,6 @@ public void testSavepointForJobWithIteration() throws Exception { iterTestCheckpointVerify[i] = 0; } - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - // Temporary directory for file state backend - final File tmpDir = folder.newFolder(); - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); final IntegerStreamSource source = new IntegerStreamSource(); IterativeStream iteration = env.addSource(source) @@ -638,20 +664,9 @@ public Integer map(Integer value) throws Exception { JobGraph jobGraph = streamGraph.getJobGraph(); - Configuration config = new Configuration(); + Configuration config = getFileBasedCheckpointsConfig(); config.addAll(jobGraph.getJobConfiguration()); config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); - final File checkpointDir = new File(tmpDir, "checkpoints"); - final File savepointDir = new File(tmpDir, "savepoints"); - - if (!checkpointDir.mkdir() || !savepointDir.mkdirs()) { - fail("Test setup failed: failed to create temporary directories."); - } - - config.setString(CheckpointingOptions.STATE_BACKEND, "filesystem"); - config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); - config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0); - config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); MiniClusterWithClientResource cluster = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() @@ -798,4 +813,65 @@ MiniClusterWithClientResource get() { .build()); } } + + private Configuration getFileBasedCheckpointsConfig(final String savepointDir) { + final Configuration config = new Configuration(); + config.setString(CheckpointingOptions.STATE_BACKEND, "filesystem"); + config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); + config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0); + config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir); + return config; + } + + private Configuration getFileBasedCheckpointsConfig() { + return getFileBasedCheckpointsConfig(savepointDir.toURI().toString()); + } + + private static Matcher hasEntropyInFileStateHandlePaths() { + return new TypeSafeDiagnosingMatcher() { + + @Override + protected boolean matchesSafely(final File savepointDir, final Description mismatchDescription) { + if (savepointDir == null) { + mismatchDescription.appendText("savepoint dir must not be null"); + return false; + } + + final List filesWithoutEntropy = listRecursively(savepointDir.toPath().resolve(EntropyInjectingTestFileSystem.ENTROPY_INJECTION_KEY)); + final Path savepointDirWithEntropy = savepointDir.toPath().resolve(EntropyInjectingTestFileSystem.ENTROPY); + final List filesWithEntropy = listRecursively(savepointDirWithEntropy); + + if (!filesWithoutEntropy.isEmpty()) { + mismatchDescription.appendText("there are savepoint files with unresolved entropy placeholders"); + return false; + } + + if (!Files.exists(savepointDirWithEntropy) || filesWithEntropy.isEmpty()) { + mismatchDescription.appendText("there are no savepoint files with added entropy"); + return false; + } + + return true; + } + + @Override + public void describeTo(final Description description) { + description.appendText("all savepoint files should have added entropy"); + } + }; + } + + private static List listRecursively(final Path dir) { + try { + if (!Files.exists(dir)) { + return Collections.emptyList(); + } else { + try (Stream files = Files.walk(dir, FileVisitOption.FOLLOW_LINKS)) { + return files.filter(Files::isRegularFile).collect(Collectors.toList()); + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } } From 1bb55d1ff271935731f8f808686e2cee69af58bd Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 15:38:38 +0100 Subject: [PATCH 082/359] [hotfix] Skip log verification for the heavy deployment e2e test --- flink-end-to-end-tests/run-nightly-tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 6d4560864f835..218700530468a 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -145,7 +145,7 @@ run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_cli run_test "SQL Client end-to-end test for Kafka 0.10" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka010.sh" run_test "SQL Client end-to-end test for modern Kafka" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka.sh" -run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" +run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" "skip_check_exceptions" run_test "ConnectedComponents iterations with high parallelism end-to-end test" "$END_TO_END_DIR/test-scripts/test_high_parallelism_iterations.sh 25" From 074fecd6f514396dcefff6cd66cab5de850f0321 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 14:56:49 +0100 Subject: [PATCH 083/359] [hotfix] Log SlotReport in SlotManager#reportSlotStatus --- .../resourcemanager/ResourceManager.java | 19 +++++++------------ .../slotmanager/SlotManager.java | 2 +- .../runtime/taskexecutor/SlotReport.java | 7 +++++++ .../runtime/taskexecutor/SlotStatus.java | 17 +++++++++++++---- 4 files changed, 28 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index cd4281e11cfa0..f90f63a2d9163 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -1145,20 +1145,15 @@ public void run() { @Override public void reportPayload(final ResourceID resourceID, final SlotReport slotReport) { - runAsync(new Runnable() { - @Override - public void run() { - log.debug("Received new slot report from TaskManager {}.", resourceID); - - final WorkerRegistration workerRegistration = taskExecutors.get(resourceID); + runAsync(() -> { + final WorkerRegistration workerRegistration = taskExecutors.get(resourceID); - if (workerRegistration == null) { - log.debug("Received slot report from TaskManager {} which is no longer registered.", resourceID); - } else { - InstanceID instanceId = workerRegistration.getInstanceID(); + if (workerRegistration == null) { + log.debug("Received slot report from TaskManager {} which is no longer registered.", resourceID); + } else { + InstanceID instanceId = workerRegistration.getInstanceID(); - slotManager.reportSlotStatus(instanceId, slotReport); - } + slotManager.reportSlotStatus(instanceId, slotReport); } }); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 2ef2b2fefcc36..1dd16934280a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -402,7 +402,7 @@ public boolean unregisterTaskManager(InstanceID instanceId) { public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) { checkInit(); - LOG.debug("Received slot report from instance {}.", instanceId); + LOG.debug("Received slot report from instance {}: {}.", instanceId, slotReport); TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java index 94ecaffa4a820..f59af99e0c591 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java @@ -53,4 +53,11 @@ public SlotReport(final Collection slotsStatus) { public Iterator iterator() { return slotsStatus.iterator(); } + + @Override + public String toString() { + return "SlotReport{" + + "slotsStatus=" + slotsStatus + + '}'; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java index a3bc4a6af4a9e..3e996b6df2764 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java @@ -51,10 +51,10 @@ public SlotStatus(SlotID slotID, ResourceProfile resourceProfile) { } public SlotStatus( - SlotID slotID, - ResourceProfile resourceProfile, - JobID jobID, - AllocationID allocationID) { + SlotID slotID, + ResourceProfile resourceProfile, + JobID jobID, + AllocationID allocationID) { this.slotID = checkNotNull(slotID, "slotID cannot be null"); this.resourceProfile = checkNotNull(resourceProfile, "profile cannot be null"); this.allocationID = allocationID; @@ -130,4 +130,13 @@ public int hashCode() { return result; } + @Override + public String toString() { + return "SlotStatus{" + + "slotID=" + slotID + + ", resourceProfile=" + resourceProfile + + ", allocationID=" + allocationID + + ", jobID=" + jobID + + '}'; + } } From 113418c726eaf31433b99f00df42604d1660bed8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 14:58:05 +0100 Subject: [PATCH 084/359] [hotfix] Fix checkstyle violations in SlotReport and SlotStatus --- .../flink/runtime/taskexecutor/SlotReport.java | 2 +- .../flink/runtime/taskexecutor/SlotStatus.java | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java index f59af99e0c591..9d8437a682841 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java @@ -34,7 +34,7 @@ public class SlotReport implements Serializable, Iterable { private static final long serialVersionUID = -3150175198722481689L; - /** The slots status of the TaskManager */ + /** The slots status of the TaskManager. */ private final Collection slotsStatus; public SlotReport() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java index 3e996b6df2764..83345c35c69a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java @@ -34,16 +34,16 @@ public class SlotStatus implements Serializable { private static final long serialVersionUID = 5099191707339664493L; - /** slotID to identify a slot */ + /** SlotID to identify a slot. */ private final SlotID slotID; - /** the resource profile of the slot */ + /** The resource profile of the slot. */ private final ResourceProfile resourceProfile; - /** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */ + /** If the slot is allocated, allocationId identify its allocation; else, allocationId is null. */ private final AllocationID allocationID; - /** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */ + /** If the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null. */ private final JobID jobID; public SlotStatus(SlotID slotID, ResourceProfile resourceProfile) { @@ -62,7 +62,7 @@ public SlotStatus( } /** - * Get the unique identification of this slot + * Get the unique identification of this slot. * * @return The slot id */ @@ -71,7 +71,7 @@ public SlotID getSlotID() { } /** - * Get the resource profile of this slot + * Get the resource profile of this slot. * * @return The resource profile */ @@ -80,7 +80,7 @@ public ResourceProfile getResourceProfile() { } /** - * Get the allocation id of this slot + * Get the allocation id of this slot. * * @return The allocation id if this slot is allocated, otherwise null */ @@ -89,7 +89,7 @@ public AllocationID getAllocationID() { } /** - * Get the job id of the slot allocated for + * Get the job id of the slot allocated for. * * @return The job id if this slot is allocated, otherwise null */ From 4c52cdbda1fb13754bad4c5ad4a7eb81ba3884f5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 14 Nov 2018 11:00:53 +0100 Subject: [PATCH 085/359] [FLINK-10877] Cleanup flink-connector-kafka pom file Remove duplicate flink-connector-kafka-base test dependencies and exclusions. --- .../flink-connector-kafka/pom.xml | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index 91b8706a035c8..dcceb7f2c14a5 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -140,28 +140,6 @@ under the License. org.apache.flink flink-connector-kafka-base_${scala.binary.version} ${project.version} - - - - org.apache.kafka - kafka_${scala.binary.version} - - - test-jar - test - - - - org.apache.flink - flink-connector-kafka-base_${scala.binary.version} - ${project.version} - - - - org.apache.kafka - kafka_${scala.binary.version} - - test-jar test From 6c21335004ce487e104bc85bd3aae211e1319a20 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 16:05:56 +0100 Subject: [PATCH 086/359] [FLINK-10856] Find latest completed checkpoint for resume from externalized checkpoint e2e test --- flink-end-to-end-tests/test-scripts/common.sh | 7 +++++++ .../test-scripts/test_resume_externalized_checkpoints.sh | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index ea2675386391c..4e6254864c127 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -656,3 +656,10 @@ function wait_for_restart_to_complete { fi done } + +function find_latest_completed_checkpoint { + local checkpoint_root_directory=$1 + # a completed checkpoint must contain the _metadata file + local checkpoint_meta_file=$(ls -d ${checkpoint_root_directory}/chk-[1-9]*/_metadata | sort -Vr | head -n1) + echo "$(dirname "${checkpoint_meta_file}")" +} diff --git a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh index c1477574d5d7f..35fe30b6b25cf 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh @@ -112,7 +112,7 @@ else fi # take the latest checkpoint -CHECKPOINT_PATH=$(ls -d $CHECKPOINT_DIR/$DATASTREAM_JOB/chk-[1-9]* | sort -Vr | head -n1) +CHECKPOINT_PATH=$(find_latest_completed_checkpoint ${CHECKPOINT_DIR}/${DATASTREAM_JOB}) if [ -z $CHECKPOINT_PATH ]; then echo "Expected an externalized checkpoint to be present, but none exists." From a2a23b233bfcbcf2c6848314453d4744478ed217 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 17:27:09 +0100 Subject: [PATCH 087/359] [hotfix] Fix checkstyle violation in ExecutionGraph --- .../org/apache/flink/runtime/executiongraph/ExecutionGraph.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 3b55e009116f5..56315e07146d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -911,7 +911,7 @@ private CompletableFuture scheduleLazy(SlotProvider slotProvider) { final CompletableFuture schedulingJobVertexFuture = ejv.scheduleAll( slotProvider, allowQueuedScheduling, - LocationPreferenceConstraint.ALL,// since it is an input vertex, the input based location preferences should be empty + LocationPreferenceConstraint.ALL, // since it is an input vertex, the input based location preferences should be empty Collections.emptySet()); schedulingFutures.add(schedulingJobVertexFuture); From f5db114940fad1b96b549801c80b312a9367a55a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 15 Nov 2018 17:27:22 +0100 Subject: [PATCH 088/359] [FLINK-10883] Failing batch jobs with NoResourceAvailableException when slot request times out Instead of failing the ExecutionGraph with a generic TimeoutException if a slot request times out, this commit changes the exception to a more meaningful NoResourceAvailableException. --- .../runtime/executiongraph/Execution.java | 14 ++++- .../minicluster/MiniClusterITCase.java | 56 ++++++++++++------- 2 files changed, 50 insertions(+), 20 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index cbf51037b8bb0..e3b501e52e837 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; @@ -427,7 +428,18 @@ public CompletableFuture scheduleForExecution( deploymentFuture.whenComplete( (Void ignored, Throwable failure) -> { if (failure != null) { - markFailed(ExceptionUtils.stripCompletionException(failure)); + final Throwable stripCompletionException = ExceptionUtils.stripCompletionException(failure); + final Throwable schedulingFailureCause; + + if (stripCompletionException instanceof TimeoutException) { + schedulingFailureCause = new NoResourceAvailableException( + "Could not allocate enough slots within timeout of " + allocationTimeout + " to run the job. " + + "Please make sure that the cluster has enough resources."); + } else { + schedulingFailureCause = stripCompletionException; + } + + markFailed(schedulingFailureCause); } }); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java index c108eaee36a38..585e1badf473c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java @@ -101,7 +101,42 @@ public void runJobWithMultipleRpcServices() throws Exception { } @Test - public void testHandleJobsWhenNotEnoughSlot() throws Exception { + public void testHandleStreamingJobsWhenNotEnoughSlot() throws Exception { + try { + setupAndRunHandleJobsWhenNotEnoughSlots(ScheduleMode.EAGER); + fail("Job should fail."); + } catch (JobExecutionException e) { + assertTrue(findThrowableWithMessage(e, "Job execution failed.").isPresent()); + assertTrue(findThrowable(e, NoResourceAvailableException.class).isPresent()); + assertTrue(findThrowableWithMessage(e, "Slots required: 2, slots allocated: 1").isPresent()); + } + } + + @Test + public void testHandleBatchJobsWhenNotEnoughSlot() throws Exception { + try { + setupAndRunHandleJobsWhenNotEnoughSlots(ScheduleMode.LAZY_FROM_SOURCES); + fail("Job should fail."); + } catch (JobExecutionException e) { + assertTrue(findThrowableWithMessage(e, "Job execution failed.").isPresent()); + assertTrue(findThrowable(e, NoResourceAvailableException.class).isPresent()); + assertTrue(findThrowableWithMessage(e, "Could not allocate enough slots").isPresent()); + } + } + + private void setupAndRunHandleJobsWhenNotEnoughSlots(ScheduleMode scheduleMode) throws Exception { + final JobVertex vertex = new JobVertex("Test Vertex"); + vertex.setParallelism(2); + vertex.setMaxParallelism(2); + vertex.setInvokableClass(BlockingNoOpInvokable.class); + + final JobGraph jobGraph = new JobGraph("Test Job", vertex); + jobGraph.setScheduleMode(scheduleMode); + + runHandleJobsWhenNotEnoughSlots(jobGraph); + } + + private void runHandleJobsWhenNotEnoughSlots(final JobGraph jobGraph) throws Exception { final Configuration configuration = getDefaultConfiguration(); configuration.setLong(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 100L); @@ -114,24 +149,7 @@ public void testHandleJobsWhenNotEnoughSlot() throws Exception { try (final MiniCluster miniCluster = new MiniCluster(cfg)) { miniCluster.start(); - final JobVertex vertex = new JobVertex("Test Vertex"); - vertex.setParallelism(2); - vertex.setMaxParallelism(2); - vertex.setInvokableClass(BlockingNoOpInvokable.class); - - final JobGraph jobGraph = new JobGraph("Test Job", vertex); - jobGraph.setScheduleMode(ScheduleMode.EAGER); - - try { - miniCluster.executeJobBlocking(jobGraph); - - fail("Job should fail."); - } catch (JobExecutionException e) { - assertTrue(findThrowableWithMessage(e, "Job execution failed.").isPresent()); - - assertTrue(findThrowable(e, NoResourceAvailableException.class).isPresent()); - assertTrue(findThrowableWithMessage(e, "Slots required: 2, slots allocated: 1").isPresent()); - } + miniCluster.executeJobBlocking(jobGraph); } } From 146f4340ed02cd2337c42b1480fd9c8b070997ad Mon Sep 17 00:00:00 2001 From: xueyu <278006819@qq.com> Date: Tue, 23 Oct 2018 00:16:48 +0800 Subject: [PATCH 089/359] [FLINK-10642] [table] Fix CodeGen split errors when maxGeneratedCodeLength equals 1 This closes #6900. --- .../flink/table/codegen/CodeGenerator.scala | 4 ++-- .../table/runtime/batch/table/CalcITCase.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index 13bf50a5ef227..b0d37370fee61 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -1061,13 +1061,13 @@ abstract class CodeGenerator( // declaration val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) - if (nullCheck) { + if (nullCheck && !expr.nullTerm.equals(NEVER_NULL) && !expr.nullTerm.equals(ALWAYS_NULL)) { reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") } reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") // assignment - if (nullCheck) { + if (nullCheck && !expr.nullTerm.equals(NEVER_NULL) && !expr.nullTerm.equals(ALWAYS_NULL)) { reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") } reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala index b161eeda43cd0..59f214bcd1ea9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala @@ -578,6 +578,22 @@ class CalcITCase( val expected = List("a,a,d,d,e,e", "x,x,z,z,z,z").mkString("\n") TestBaseUtils.compareResultAsText(results.asJava, expected) } + + @Test + def testSplitFieldsOnCustomType(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env, config) + tEnv.getConfig.setMaxGeneratedCodeLength(1) // splits fields + + val ds = CollectionDataSets.getCustomTypeDataSet(env) + val filterDs = ds.toTable(tEnv, 'myInt as 'i, 'myLong as 'l, 'myString as 's) + .filter('s.like("%a%") && 's.charLength() > 12) + .select('i, 'l, 's.charLength()) + + val expected = "3,3,25\n" + "3,5,14\n" + val results = filterDs.toDataSet[Row].collect() + TestBaseUtils.compareResultAsText(results.asJava, expected) + } } object CalcITCase { From d6788bf47712177f9a2adf9e7e941a0b8110e818 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 15 Nov 2018 15:24:29 +0100 Subject: [PATCH 090/359] [hotfix][kafka][docs] Split long lines in kafka.md Long lines are not diff/conflict resolution friendly, while md ignores new lines, so this change has no visible effect for the user. --- docs/dev/connectors/kafka.md | 44 +++++++++++++++++++++++++----------- 1 file changed, 31 insertions(+), 13 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index f81080a4b5315..46fc24ebe99fc 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -86,7 +86,11 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is FlinkKafkaConsumer
FlinkKafkaProducer >= 1.0.0 - This Kafka connector attempts to track the latest version of the Kafka client. The version of the client it uses may change between Flink releases. Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. However for Kafka 0.11.x and 0.10.x versions, we recommend using dedicated flink-connector-kafka-0.11 and link-connector-kafka-0.10 respectively. + This universal Kafka connector attempts to track the latest version of the Kafka client. + The version of the client it uses may change between Flink releases. + Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. + However for Kafka 0.11.x and 0.10.x versions, we recommend using dedicated + flink-connector-kafka-0.11 and link-connector-kafka-0.10 respectively. @@ -101,7 +105,8 @@ Then, import the connector in your maven project:
{% endhighlight %} -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}}/dev/linking.html). +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}}/dev/linking.html). ## Installing Apache Kafka @@ -110,13 +115,17 @@ Note that the streaming connectors are currently not part of the binary distribu ## Kafka 1.0.0+ Connector -Starting with Flink 1.7, there is a new Kafka connector that does not track a specific Kafka major version. Rather, it tracks the latest version of Kafka at the time of the Flink release. +Starting with Flink 1.7, there is a new Kafka connector that does not track a specific Kafka major version. +Rather, it tracks the latest version of Kafka at the time of the Flink release. -If your Kafka broker version is 1.0.0 or newer, you should use this Kafka connector. If you use an older version of Kafka (0.11, 0.10, 0.9, or 0.8), you should use the connector corresponding to the broker version. +If your Kafka broker version is 1.0.0 or newer, you should use this Kafka connector. +If you use an older version of Kafka (0.11, 0.10, 0.9, or 0.8), you should use the connector corresponding to the broker version. ### Compatibility -The modern Kafka connector is compatible with older and newer Kafka brokers through the compatibility guarantees of the Kafka client API and broker. The modern Kafka connector is compatible with broker versions 0.11.0 or later, depending on the features used. For details on Kafka compatibility, please refer to the [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). +The modern Kafka connector is compatible with older and newer Kafka brokers through the compatibility guarantees of the Kafka client API and broker. +The modern Kafka connector is compatible with broker versions 0.11.0 or later, depending on the features used. +For details on Kafka compatibility, please refer to the [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). ### Usage @@ -130,11 +139,13 @@ The use of the modern Kafka connector add a dependency to it: {% endhighlight %} -Then instantiate the new source (`FlinkKafkaConsumer`) and sink (`FlinkKafkaProducer`). The API is the backwards compatible with the older Kafka connectors. +Then instantiate the new source (`FlinkKafkaConsumer`) and sink (`FlinkKafkaProducer`). +The API is the backwards compatible with the older Kafka connectors. ## Kafka Consumer -Flink's Kafka consumer is called `FlinkKafkaConsumer08` (or 09 for Kafka 0.9.0.x versions, etc. or just `FlinkKafkaConsumer` for Kafka >= 1.0.0 versions). It provides access to one or more Kafka topics. +Flink's Kafka consumer is called `FlinkKafkaConsumer08` (or 09 for Kafka 0.9.0.x versions, etc. +or just `FlinkKafkaConsumer` for Kafka >= 1.0.0 versions). It provides access to one or more Kafka topics. The constructor accepts the following arguments: @@ -516,7 +527,10 @@ the `Watermark getCurrentWatermark()` (for periodic) or the `Watermark checkAndGetNextWatermark(T lastElement, long extractedTimestamp)` (for punctuated) is called to determine if a new watermark should be emitted and with which timestamp. -**Note**: If a watermark assigner depends on records read from Kafka to advance its watermarks (which is commonly the case), all topics and partitions need to have a continuous stream of records. Otherwise, the watermarks of the whole application cannot advance and all time-based operations, such as time windows or functions with timers, cannot make progress. A single idle Kafka partition causes this behavior. +**Note**: If a watermark assigner depends on records read from Kafka to advance its watermarks +(which is commonly the case), all topics and partitions need to have a continuous stream of records. +Otherwise, the watermarks of the whole application cannot advance and all time-based operations, +such as time windows or functions with timers, cannot make progress. A single idle Kafka partition causes this behavior. A Flink improvement is planned to prevent this from happening (see [FLINK-5479: Per-partition watermarks in FlinkKafkaConsumer should consider idle partitions]( https://issues.apache.org/jira/browse/FLINK-5479)). @@ -729,7 +743,8 @@ application before first checkpoint completes, by factor larger then `FlinkKafka ## Using Kafka timestamps and Flink event time in Kafka 0.10 -Since Apache Kafka 0.10+, Kafka's messages can carry [timestamps](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message), indicating +Since Apache Kafka 0.10+, Kafka's messages can carry +[timestamps](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message), indicating the time the event has occurred (see ["event time" in Apache Flink](../event_time.html)) or the time when the message has been written to the Kafka broker. @@ -789,17 +804,20 @@ Flink provides first-class support through the Kafka connector to authenticate t configured for Kerberos. Simply configure Flink in `flink-conf.yaml` to enable Kerberos authentication for Kafka like so: 1. Configure Kerberos credentials by setting the following - - - `security.kerberos.login.use-ticket-cache`: By default, this is `true` and Flink will attempt to use Kerberos credentials in ticket caches managed by `kinit`. - Note that when using the Kafka connector in Flink jobs deployed on YARN, Kerberos authorization using ticket caches will not work. This is also the case when deploying using Mesos, as authorization using ticket cache is not supported for Mesos deployments. + - `security.kerberos.login.use-ticket-cache`: By default, this is `true` and Flink will attempt to use Kerberos credentials in ticket caches managed by `kinit`. + Note that when using the Kafka connector in Flink jobs deployed on YARN, Kerberos authorization using ticket caches will not work. + This is also the case when deploying using Mesos, as authorization using ticket cache is not supported for Mesos deployments. - `security.kerberos.login.keytab` and `security.kerberos.login.principal`: To use Kerberos keytabs instead, set values for both of these properties. 2. Append `KafkaClient` to `security.kerberos.login.contexts`: This tells Flink to provide the configured Kerberos credentials to the Kafka login context to be used for Kafka authentication. -Once Kerberos-based Flink security is enabled, you can authenticate to Kafka with either the Flink Kafka Consumer or Producer by simply including the following two settings in the provided properties configuration that is passed to the internal Kafka client: +Once Kerberos-based Flink security is enabled, you can authenticate to Kafka with either the Flink Kafka Consumer or Producer +by simply including the following two settings in the provided properties configuration that is passed to the internal Kafka client: - Set `security.protocol` to `SASL_PLAINTEXT` (default `NONE`): The protocol used to communicate to Kafka brokers. When using standalone Flink deployment, you can also use `SASL_SSL`; please see how to configure the Kafka client for SSL [here](https://kafka.apache.org/documentation/#security_configclients). -- Set `sasl.kerberos.service.name` to `kafka` (default `kafka`): The value for this should match the `sasl.kerberos.service.name` used for Kafka broker configurations. A mismatch in service name between client and server configuration will cause the authentication to fail. +- Set `sasl.kerberos.service.name` to `kafka` (default `kafka`): The value for this should match the `sasl.kerberos.service.name` used for Kafka broker configurations. +A mismatch in service name between client and server configuration will cause the authentication to fail. For more information on Flink configuration for Kerberos security, please see [here]({{ site.baseurl}}/ops/config.html). You can also find [here]({{ site.baseurl}}/ops/security-kerberos.html) further details on how Flink internally setups Kerberos-based security. From 89b3d974828200c17377af07540a6e8bae717905 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 15 Nov 2018 15:11:20 +0100 Subject: [PATCH 091/359] [hotfix][kafka][docs] Couple of minor fixes in Kafka 2.0 connector documentation --- docs/dev/connectors/kafka.md | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 46fc24ebe99fc..bd4d49bffa5d7 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -73,7 +73,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is This connector supports
Kafka messages with timestamps both for producing and consuming. - flink-connector-kafka-0.11_2.11 + flink-connector-kafka-0.11{{ site.scala_version_suffix }} 1.4.0 FlinkKafkaConsumer011
FlinkKafkaProducer011 @@ -81,7 +81,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is Since 0.11.x Kafka does not support scala 2.10. This connector supports Kafka transactional messaging to provide exactly once semantic for the producer. - flink-connector-kafka_2.11 + flink-connector-kafka{{ site.scala_version_suffix }} 1.7.0 FlinkKafkaConsumer
FlinkKafkaProducer @@ -90,7 +90,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is The version of the client it uses may change between Flink releases. Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. However for Kafka 0.11.x and 0.10.x versions, we recommend using dedicated - flink-connector-kafka-0.11 and link-connector-kafka-0.10 respectively. + flink-connector-kafka-0.11{{ site.scala_version_suffix }} and link-connector-kafka-0.10{{ site.scala_version_suffix }} respectively. @@ -115,7 +115,7 @@ See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/lin ## Kafka 1.0.0+ Connector -Starting with Flink 1.7, there is a new Kafka connector that does not track a specific Kafka major version. +Starting with Flink 1.7, there is a new universal Kafka connector that does not track a specific Kafka major version. Rather, it tracks the latest version of Kafka at the time of the Flink release. If your Kafka broker version is 1.0.0 or newer, you should use this Kafka connector. @@ -123,13 +123,13 @@ If you use an older version of Kafka (0.11, 0.10, 0.9, or 0.8), you should use t ### Compatibility -The modern Kafka connector is compatible with older and newer Kafka brokers through the compatibility guarantees of the Kafka client API and broker. -The modern Kafka connector is compatible with broker versions 0.11.0 or later, depending on the features used. +The universal Kafka connector is compatible with older and newer Kafka brokers through the compatibility guarantees of the Kafka client API and broker. +It is compatible with broker versions 0.11.0 or newer, depending on the features used. For details on Kafka compatibility, please refer to the [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). ### Usage -The use of the modern Kafka connector add a dependency to it: +To use the universal Kafka connector add a dependency to it: {% highlight xml %} @@ -140,7 +140,8 @@ The use of the modern Kafka connector add a dependency to it: {% endhighlight %} Then instantiate the new source (`FlinkKafkaConsumer`) and sink (`FlinkKafkaProducer`). -The API is the backwards compatible with the older Kafka connectors. +The API is backward compatible with the Kafka 0.11 connector, +except of dropping specific Kafka version from the module and class names. ## Kafka Consumer From d02af7eda35edbfe2fe988e4573ad5ed6c79843f Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 15 Nov 2018 15:15:03 +0100 Subject: [PATCH 092/359] [FLINK-10900][kafka][docs] Mark universal Kafka connector as beta Because of frequent deadlocks and other failures in tests the connector is marked as beta. Those errors are probably only test issues or Kafka bugs but as for now it's not confirmed. --- docs/dev/connectors/kafka.md | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index bd4d49bffa5d7..0630c6ec7d6c7 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -86,12 +86,20 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is FlinkKafkaConsumer
FlinkKafkaProducer >= 1.0.0 - This universal Kafka connector attempts to track the latest version of the Kafka client. + + This universal Kafka connector attempts to track the latest version of the Kafka client. The version of the client it uses may change between Flink releases. Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. However for Kafka 0.11.x and 0.10.x versions, we recommend using dedicated - flink-connector-kafka-0.11{{ site.scala_version_suffix }} and link-connector-kafka-0.10{{ site.scala_version_suffix }} respectively. - + flink-connector-kafka-0.11{{ site.scala_version_suffix }} and link-connector-kafka-0.10{{ site.scala_version_suffix }} respectively. +
+ Attention: as of Flink 1.7 the universal Kafka connector is considered to be + in a BETA status and might not be as stable as the 0.11 connector. + In case of problems with the universal connector, you can try to use flink-connector-kafka-0.11{{ site.scala_version_suffix }} + which should be compatible with all of the Kafka versions starting from 0.11. +
+ + From d2c2773972fa0b4c5a348d2f885b6c9956432c18 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 16 Nov 2018 09:23:28 +0100 Subject: [PATCH 093/359] [FLINK-10843] [connectors] Change Kafka table factory version '2.0' to 'universal' This closes #7087. --- docs/dev/table/connect.md | 27 +++++++++++-------- .../table/descriptors/KafkaValidator.java | 17 ++---------- .../kafka/KafkaTableSourceSinkFactory.java | 2 +- .../KafkaTableSourceSinkFactoryTest.java | 2 +- .../test-scripts/kafka_sql_common.sh | 3 ++- .../test-scripts/test_sql_client.sh | 8 +++++- .../test-scripts/test_sql_client_kafka.sh | 2 +- .../test-scripts/test_sql_client_kafka010.sh | 2 +- .../test_sql_client_kafka_common.sh | 10 +++++-- 9 files changed, 39 insertions(+), 34 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index 5f1112706cee7..effd913707ea6 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -40,14 +40,15 @@ The following table list all available connectors and formats. Their mutual comp ### Connectors -| Name | Version | Maven dependency | SQL Client JAR | -| :---------------- | :------------ | :--------------------------- | :----------------------| -| Filesystem | | Built-in | Built-in | -| Elasticsearch | 6 | `flink-connector-elasticsearch6` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-elasticsearch6{{site.scala_version_suffix}}/{{site.version}}/flink-connector-elasticsearch6{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | -| Apache Kafka | 0.8 | `flink-connector-kafka-0.8` | Not available | -| Apache Kafka | 0.9 | `flink-connector-kafka-0.9` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.9{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.9{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | -| Apache Kafka | 0.10 | `flink-connector-kafka-0.10` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.10{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.10{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | -| Apache Kafka | 0.11 | `flink-connector-kafka-0.11` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.11{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.11{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | +| Name | Version | Maven dependency | SQL Client JAR | +| :---------------- | :------------------ | :--------------------------- | :----------------------| +| Filesystem | | Built-in | Built-in | +| Elasticsearch | 6 | `flink-connector-elasticsearch6` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-elasticsearch6{{site.scala_version_suffix}}/{{site.version}}/flink-connector-elasticsearch6{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | +| Apache Kafka | 0.8 | `flink-connector-kafka-0.8` | Not available | +| Apache Kafka | 0.9 | `flink-connector-kafka-0.9` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.9{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.9{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | +| Apache Kafka | 0.10 | `flink-connector-kafka-0.10` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.10{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.10{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | +| Apache Kafka | 0.11 | `flink-connector-kafka-0.11` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.11{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.11{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | +| Apache Kafka | 0.11+ (`universal`) | `flink-connector-kafka` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | ### Formats @@ -524,7 +525,8 @@ The Kafka connector allows for reading and writing from and to an Apache Kafka t {% highlight java %} .connect( new Kafka() - .version("0.11") // required: valid connector versions are "0.8", "0.9", "0.10", and "0.11" + .version("0.11") // required: valid connector versions are + // "0.8", "0.9", "0.10", "0.11", and "universal" .topic("...") // required: topic name from which the table is read // optional: connector specific properties @@ -549,7 +551,8 @@ The Kafka connector allows for reading and writing from and to an Apache Kafka t {% highlight yaml %} connector: type: kafka - version: "0.11" # required: valid connector versions are "0.8", "0.9", "0.10", and "0.11" + version: "0.11" # required: valid connector versions are + # "0.8", "0.9", "0.10", "0.11", and "universal" topic: ... # required: topic name from which the table is read properties: # optional: connector specific properties @@ -583,7 +586,9 @@ connector: **Consistency guarantees:** By default, a Kafka sink ingests data with at-least-once guarantees into a Kafka topic if the query is executed with [checkpointing enabled]({{ site.baseurl }}/dev/stream/state/checkpointing.html#enabling-and-configuring-checkpointing). -**Kafka 0.10+ Timestamps:** Since Kafka 0.10, Kafka messages have a timestamp as metadata that specifies when the record was written into the Kafka topic. These timestamps can be used for a [rowtime attribute](connect.html#defining-the-schema) by selecting `timestamps: from-source` in YAML and `timestampsFromSource()` in Java/Scala respectively. +**Kafka 0.10+ Timestamps:** Since Kafka 0.10, Kafka messages have a timestamp as metadata that specifies when the record was written into the Kafka topic. These timestamps can be used for a [rowtime attribute](connect.html#defining-the-schema) by selecting `timestamps: from-source` in YAML and `timestampsFromSource()` in Java/Scala respectively. + +**Kafka 0.11+ Versioning:** Since Flink 1.7, the Kafka connector definition should be independent of a hard-coded Kafka version. Use the connector version `universal` as a wildcard for Flink's Kafka connector that is compatible with all Kafka versions starting from 0.11. Make sure to add the version-specific Kafka dependency. In addition, a corresponding format needs to be specified for reading and writing rows from and to Kafka. diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java index 86d7ef6d35ba3..0dbbbeb348a01 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java @@ -21,9 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; -import java.util.Arrays; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.function.Consumer; @@ -40,7 +38,7 @@ public class KafkaValidator extends ConnectorDescriptorValidator { public static final String CONNECTOR_VERSION_VALUE_09 = "0.9"; public static final String CONNECTOR_VERSION_VALUE_010 = "0.10"; public static final String CONNECTOR_VERSION_VALUE_011 = "0.11"; - public static final String CONNECTOR_VERSION_VALUE_20 = "2.0"; + public static final String CONNECTOR_VERSION_VALUE_UNIVERSAL = "universal"; public static final String CONNECTOR_TOPIC = "connector.topic"; public static final String CONNECTOR_STARTUP_MODE = "connector.startup-mode"; public static final String CONNECTOR_STARTUP_MODE_VALUE_EARLIEST = "earliest-offset"; @@ -64,7 +62,7 @@ public void validate(DescriptorProperties properties) { super.validate(properties); properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_KAFKA, false); - validateVersion(properties); + properties.validateString(CONNECTOR_TOPIC, false, 1, Integer.MAX_VALUE); validateStartupMode(properties); @@ -73,17 +71,6 @@ public void validate(DescriptorProperties properties) { validateSinkPartitioner(properties); } - private void validateVersion(DescriptorProperties properties) { - final List versions = Arrays.asList( - CONNECTOR_VERSION_VALUE_08, - CONNECTOR_VERSION_VALUE_09, - CONNECTOR_VERSION_VALUE_010, - CONNECTOR_VERSION_VALUE_011, - CONNECTOR_VERSION_VALUE_20); - properties.validateEnumValues(CONNECTOR_VERSION, false, versions); - properties.validateString(CONNECTOR_TOPIC, false, 1, Integer.MAX_VALUE); - } - private void validateStartupMode(DescriptorProperties properties) { final Map> specificOffsetValidators = new HashMap<>(); specificOffsetValidators.put( diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactory.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactory.java index b0dfc54e67171..2b498674a89df 100644 --- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactory.java +++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactory.java @@ -39,7 +39,7 @@ public class KafkaTableSourceSinkFactory extends KafkaTableSourceSinkFactoryBase @Override protected String kafkaVersion() { - return KafkaValidator.CONNECTOR_VERSION_VALUE_20; + return KafkaValidator.CONNECTOR_VERSION_VALUE_UNIVERSAL; } @Override diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTest.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTest.java index 5043880b21e14..4d843bcd82e05 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTest.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTest.java @@ -40,7 +40,7 @@ public class KafkaTableSourceSinkFactoryTest extends KafkaTableSourceSinkFactory @Override protected String getKafkaVersion() { - return KafkaValidator.CONNECTOR_VERSION_VALUE_20; + return KafkaValidator.CONNECTOR_VERSION_VALUE_UNIVERSAL; } @Override diff --git a/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh b/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh index 3e95b405b1056..468f0587db757 100644 --- a/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh +++ b/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh @@ -21,6 +21,7 @@ KAFKA_CONNECTOR_VERSION="$1" KAFKA_VERSION="$2" CONFLUENT_VERSION="$3" CONFLUENT_MAJOR_VERSION="$4" +KAFKA_SQL_VERSION="$5" source "$(dirname "$0")"/kafka-common.sh $2 $3 $4 @@ -64,7 +65,7 @@ function get_kafka_json_source_schema { type: ROW connector: type: kafka - version: "$KAFKA_CONNECTOR_VERSION" + version: "$KAFKA_SQL_VERSION" topic: $topicName startup-mode: earliest-offset properties: diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index 74192b40dfb21..5dd68838ba7ae 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -23,9 +23,15 @@ KAFKA_CONNECTOR_VERSION="2.0" KAFKA_VERSION="2.0.0" CONFLUENT_VERSION="5.0.0" CONFLUENT_MAJOR_VERSION="5.0" +KAFKA_SQL_VERSION="universal" source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka_sql_common.sh $KAFKA_CONNECTOR_VERSION $KAFKA_VERSION $CONFLUENT_VERSION $CONFLUENT_MAJOR_VERSION +source "$(dirname "$0")"/kafka_sql_common.sh \ + $KAFKA_CONNECTOR_VERSION \ + $KAFKA_VERSION \ + $CONFLUENT_VERSION \ + $CONFLUENT_MAJOR_VERSION \ + $KAFKA_SQL_VERSION source "$(dirname "$0")"/elasticsearch-common.sh SQL_TOOLBOX_JAR=$END_TO_END_DIR/flink-sql-client-test/target/SqlToolbox.jar diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh index a05dc050a601a..94e89a2b1e854 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh @@ -19,4 +19,4 @@ set -Eeuo pipefail -source "$(dirname "$0")"/test_sql_client_kafka_common.sh 2.0 2.0.0 5.0.0 5.0 "kafka" +source "$(dirname "$0")"/test_sql_client_kafka_common.sh 2.0 2.0.0 5.0.0 5.0 "kafka" "universal" diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka010.sh b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka010.sh index c710abc72bc07..66bef6623a156 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka010.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka010.sh @@ -19,4 +19,4 @@ set -Eeuo pipefail -source "$(dirname "$0")"/test_sql_client_kafka_common.sh 0.10 0.10.2.0 3.2.0 3.2 "kafka-0.10" +source "$(dirname "$0")"/test_sql_client_kafka_common.sh 0.10 0.10.2.0 3.2.0 3.2 "kafka-0.10" "0.10" diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka_common.sh b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka_common.sh index 149c86f570ef5..08ed59b419c22 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka_common.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka_common.sh @@ -22,9 +22,15 @@ KAFKA_VERSION="$2" CONFLUENT_VERSION="$3" CONFLUENT_MAJOR_VERSION="$4" KAFKA_SQL_JAR="$5" +KAFKA_SQL_VERSION="$6" source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka_sql_common.sh $KAFKA_CONNECTOR_VERSION $KAFKA_VERSION $CONFLUENT_VERSION $CONFLUENT_MAJOR_VERSION +source "$(dirname "$0")"/kafka_sql_common.sh \ + $KAFKA_CONNECTOR_VERSION \ + $KAFKA_VERSION \ + $CONFLUENT_VERSION \ + $CONFLUENT_MAJOR_VERSION \ + $KAFKA_SQL_VERSION ################################################################################ # Prepare connectors @@ -98,7 +104,7 @@ cat >> $SQL_CONF << EOF type: BIGINT connector: type: kafka - version: "$KAFKA_CONNECTOR_VERSION" + version: "$KAFKA_SQL_VERSION" topic: test-avro startup-mode: earliest-offset properties: From 1891949d2584e67a5edb878eac62268c447fdc1f Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 15 Nov 2018 22:15:51 +0800 Subject: [PATCH 094/359] [FLINK-10872] [e2e] Extend SQL Client end-to-end to for Kafka 0.11 connector This closes #7100. --- .../flink-sql-client-test/pom.xml | 7 +++--- flink-end-to-end-tests/run-nightly-tests.sh | 1 + .../test-scripts/test_sql_client_kafka011.sh | 22 +++++++++++++++++++ 3 files changed, 26 insertions(+), 4 deletions(-) create mode 100755 flink-end-to-end-tests/test-scripts/test_sql_client_kafka011.sh diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml index 7710d01dda9bd..d744065a7b459 100644 --- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml +++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml @@ -186,19 +186,18 @@ under the License. org.apache.flink - flink-connector-kafka_${scala.binary.version} + flink-connector-kafka-0.11_${scala.binary.version} ${project.version} sql-jar jar - + org.apache.flink flink-connector-elasticsearch6_${scala.binary.version} diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 218700530468a..24d77c3841fcc 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -143,6 +143,7 @@ run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scrip run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_client.sh" run_test "SQL Client end-to-end test for Kafka 0.10" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka010.sh" +run_test "SQL Client end-to-end test for Kafka 0.11" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka011.sh" run_test "SQL Client end-to-end test for modern Kafka" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka.sh" run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" "skip_check_exceptions" diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka011.sh b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka011.sh new file mode 100755 index 0000000000000..e8ab9d0b29d10 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka011.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +set -Eeuo pipefail + +source "$(dirname "$0")"/test_sql_client_kafka_common.sh 0.11 0.11.0.2 3.2.0 3.2 "kafka-0.11" "0.11" From 7f305206a85c8e723dbe65d0cbd8981d9ef91ef9 Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 15 Nov 2018 14:26:34 +0800 Subject: [PATCH 095/359] [FLINK-10891] Upgrade Kafka client version to 2.0.1 --- flink-connectors/flink-connector-kafka/pom.xml | 2 +- flink-end-to-end-tests/flink-sql-client-test/pom.xml | 2 +- flink-end-to-end-tests/test-scripts/test_sql_client.sh | 2 +- flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh | 2 +- flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index dcceb7f2c14a5..8619980fe8f0c 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -36,7 +36,7 @@ under the License. jar - 2.0.0 + 2.0.1 diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml index d744065a7b459..5edd8e35ae7a3 100644 --- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml +++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml @@ -114,7 +114,7 @@ under the License. as we neither access nor package the kafka dependencies --> org.apache.kafka kafka-clients - 2.0.0 + 2.0.1
diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index 5dd68838ba7ae..8ace250a31900 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -20,7 +20,7 @@ set -Eeuo pipefail KAFKA_CONNECTOR_VERSION="2.0" -KAFKA_VERSION="2.0.0" +KAFKA_VERSION="2.0.1" CONFLUENT_VERSION="5.0.0" CONFLUENT_MAJOR_VERSION="5.0" KAFKA_SQL_VERSION="universal" diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh index 94e89a2b1e854..0941cf200f1ec 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client_kafka.sh @@ -19,4 +19,4 @@ set -Eeuo pipefail -source "$(dirname "$0")"/test_sql_client_kafka_common.sh 2.0 2.0.0 5.0.0 5.0 "kafka" "universal" +source "$(dirname "$0")"/test_sql_client_kafka_common.sh 2.0 2.0.1 5.0.0 5.0 "kafka" "universal" diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh index 044d223798811..ff36cf1d77808 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh @@ -20,6 +20,6 @@ set -Eeuo pipefail source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh 2.0.0 5.0.0 5.0 +source "$(dirname "$0")"/kafka-common.sh 2.0.1 5.0.0 5.0 source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/KafkaExample.jar From 6a15de185d6b1c6e8493240e96d00ab2e74c8f7c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Nov 2018 18:51:23 +0100 Subject: [PATCH 096/359] [hotfix] Add release note that Flink's Scala shell does not work with Scala 2.12 --- docs/release-notes/flink-1.7.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/release-notes/flink-1.7.md b/docs/release-notes/flink-1.7.md index ea6840e65c140..f9e7425601a42 100644 --- a/docs/release-notes/flink-1.7.md +++ b/docs/release-notes/flink-1.7.md @@ -20,6 +20,14 @@ specific language governing permissions and limitations under the License. --> -These release notes discuss important aspects, such as configuration, behavior, or dependencies, that changed between Flink 1.6 and Flink 1.7. Please read these notes carefully if you are planning to upgrade your Flink version to 1.7. +These release notes discuss important aspects, such as configuration, behavior, or dependencies, that changed between Flink 1.6 and Flink 1.7. Please read these notes carefully if you are planning to upgrade your Flink version to 1.7. + + +### Scala shell does not work with Scala 2.12 + +Flink's Scala shell does not work with Scala 2.12. +Therefore, the module `flink-scala-shell` is not being released for Scala 2.12. + +See [FLINK-10911](https://issues.apache.org/jira/browse/FLINK-10911) for more details. {% top %} From a66b2676fa4576f0d1e20c82612fb6b71a32b76c Mon Sep 17 00:00:00 2001 From: kkloudas Date: Tue, 6 Nov 2018 18:02:10 +0100 Subject: [PATCH 097/359] [FLINK-10531][e2e] Fix unstable TTL end-to-end test. --- .../tests/DataStreamStateTTLTestProgram.java | 21 ++++- .../tests/MonotonicTTLTimeProvider.java | 73 ++++++++++++++ .../streaming/tests/StubStateBackend.java | 94 +++++++++++++++++++ .../flink/streaming/tests/TtlStateUpdate.java | 3 + .../streaming/tests/TtlStateUpdateSource.java | 3 + .../tests/TtlVerifyUpdateFunction.java | 74 ++++++--------- .../verify/AbstractTtlStateVerifier.java | 15 +-- .../verify/TtlAggregatingStateVerifier.java | 6 +- .../tests/verify/TtlFoldingStateVerifier.java | 6 +- .../tests/verify/TtlListStateVerifier.java | 2 +- .../tests/verify/TtlMapStateVerifier.java | 2 +- .../verify/TtlReducingStateVerifier.java | 6 +- .../tests/verify/TtlUpdateContext.java | 35 ++++--- .../tests/verify/TtlValueStateVerifier.java | 2 +- .../tests/verify/TtlVerificationContext.java | 8 +- .../streaming/tests/verify/ValueWithTs.java | 31 +++--- 16 files changed, 276 insertions(+), 105 deletions(-) create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.java create mode 100644 flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java index 3b2e4746b62df..1a572f314c52f 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; @@ -74,6 +75,8 @@ public static void main(String[] args) throws Exception { setupEnvironment(env, pt); + final MonotonicTTLTimeProvider ttlTimeProvider = setBackendWithCustomTTLTimeProvider(env); + int keySpace = pt.getInt(UPDATE_GENERATOR_SRC_KEYSPACE.key(), UPDATE_GENERATOR_SRC_KEYSPACE.defaultValue()); long sleepAfterElements = pt.getLong(UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS.key(), UPDATE_GENERATOR_SRC_SLEEP_AFTER_ELEMENTS.defaultValue()); @@ -90,11 +93,27 @@ public static void main(String[] args) throws Exception { .addSource(new TtlStateUpdateSource(keySpace, sleepAfterElements, sleepTime)) .name("TtlStateUpdateSource") .keyBy(TtlStateUpdate::getKey) - .flatMap(new TtlVerifyUpdateFunction(ttlConfig, reportStatAfterUpdatesNum)) + .flatMap(new TtlVerifyUpdateFunction(ttlConfig, ttlTimeProvider, reportStatAfterUpdatesNum)) .name("TtlVerifyUpdateFunction") .addSink(new PrintSinkFunction<>()) .name("PrintFailedVerifications"); env.execute("State TTL test job"); } + + /** + * Sets the state backend to a new {@link StubStateBackend} which has a {@link MonotonicTTLTimeProvider}. + * + * @param env The {@link StreamExecutionEnvironment} of the job. + * @return The {@link MonotonicTTLTimeProvider}. + */ + private static MonotonicTTLTimeProvider setBackendWithCustomTTLTimeProvider(StreamExecutionEnvironment env) { + final MonotonicTTLTimeProvider ttlTimeProvider = new MonotonicTTLTimeProvider(); + + final StateBackend configuredBackend = env.getStateBackend(); + final StateBackend stubBackend = new StubStateBackend(configuredBackend, ttlTimeProvider); + env.setStateBackend(stubBackend); + + return ttlTimeProvider; + } } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.java new file mode 100644 index 0000000000000..0b5637d36c400 --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/MonotonicTTLTimeProvider.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.streaming.tests; + +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.Serializable; + +/** + * A stub implementation of a {@link TtlTimeProvider} which guarantees that + * processing time increases monotonically. + */ +@NotThreadSafe +final class MonotonicTTLTimeProvider implements TtlTimeProvider, Serializable { + + private static final long serialVersionUID = 1L; + + /* + * The following variables are static because the whole TTLTimeProvider will go + * through serialization and, eventually, the state backend and the task executing + * the TtlVerifyUpdateFunction will have different instances of it. + * + * If these were not static, then the TtlVerifyUpdateFunction would e.g. freeze + * the time, but the backend would not be notified about it, resulting in inconsistent + * state. + * + * If the number of task slots per TM changes, then we may need to add also synchronization. + */ + + private static boolean timeIsFrozen = false; + + private static long lastReturnedProcessingTime = Long.MIN_VALUE; + + @Override + public long currentTimestamp() { + if (timeIsFrozen && lastReturnedProcessingTime != Long.MIN_VALUE) { + return lastReturnedProcessingTime; + } + + timeIsFrozen = true; + + final long currentProcessingTime = System.currentTimeMillis(); + if (currentProcessingTime < lastReturnedProcessingTime) { + return lastReturnedProcessingTime; + } + + lastReturnedProcessingTime = currentProcessingTime; + return lastReturnedProcessingTime; + } + + long unfreezeTime() { + timeIsFrozen = false; + return lastReturnedProcessingTime; + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java new file mode 100644 index 0000000000000..b93fa362d7fec --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/StubStateBackend.java @@ -0,0 +1,94 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.ttl.TtlTimeProvider; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A stub implementation of the {@link StateBackend} that allows the use of + * a custom {@link TtlTimeProvider}. + */ +final class StubStateBackend implements StateBackend { + + private static final long serialVersionUID = 1L; + + private final TtlTimeProvider ttlTimeProvider; + + private final StateBackend backend; + + StubStateBackend(final StateBackend wrappedBackend, final TtlTimeProvider ttlTimeProvider) { + this.backend = checkNotNull(wrappedBackend); + this.ttlTimeProvider = checkNotNull(ttlTimeProvider); + } + + @Override + public CompletedCheckpointStorageLocation resolveCheckpoint(String externalPointer) throws IOException { + return backend.resolveCheckpoint(externalPointer); + } + + @Override + public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException { + return backend.createCheckpointStorage(jobId); + } + + @Override + public AbstractKeyedStateBackend createKeyedStateBackend( + Environment env, + JobID jobID, + String operatorIdentifier, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + TaskKvStateRegistry kvStateRegistry, + TtlTimeProvider ttlTimeProvider, + MetricGroup metricGroup) throws Exception { + + return backend.createKeyedStateBackend( + env, + jobID, + operatorIdentifier, + keySerializer, + numberOfKeyGroups, + keyGroupRange, + kvStateRegistry, + this.ttlTimeProvider, + metricGroup + ); + } + + @Override + public OperatorStateBackend createOperatorStateBackend(Environment env, String operatorIdentifier) throws Exception { + return backend.createOperatorStateBackend(env, operatorIdentifier); + } +} diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java index e89b544cc669a..20e21ead233bf 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdate.java @@ -25,6 +25,9 @@ /** Randomly generated keyed state updates per state type. */ class TtlStateUpdate implements Serializable { + + private static final long serialVersionUID = 1L; + private final int key; @Nonnull diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java index 6aff14e1cd3df..7404adf7771ff 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlStateUpdateSource.java @@ -33,6 +33,9 @@ * and waits for {@code sleepTime} to continue generation. */ class TtlStateUpdateSource extends RichParallelSourceFunction { + + private static final long serialVersionUID = 1L; + private final int maxKey; private final long sleepAfterElements; private final long sleepTime; diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java index 3cfb0e2d86ec8..250041d0e34e4 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlVerifyUpdateFunction.java @@ -33,7 +33,6 @@ import org.apache.flink.streaming.tests.verify.TtlVerificationContext; import org.apache.flink.streaming.tests.verify.ValueWithTs; import org.apache.flink.util.Collector; -import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,12 +40,14 @@ import javax.annotation.Nonnull; import java.io.Serializable; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + /** * Update state with TTL for each verifier. * @@ -62,21 +63,26 @@ * - verifies last update against previous updates * - emits verification context in case of failure */ -class TtlVerifyUpdateFunction - extends RichFlatMapFunction implements CheckpointedFunction { +class TtlVerifyUpdateFunction extends RichFlatMapFunction implements CheckpointedFunction { + + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(TtlVerifyUpdateFunction.class); @Nonnull private final StateTtlConfig ttlConfig; - private final long ttl; + private final MonotonicTTLTimeProvider ttlTimeProvider; private final UpdateStat stat; private transient Map states; private transient Map>> prevUpdatesByVerifierId; - TtlVerifyUpdateFunction(@Nonnull StateTtlConfig ttlConfig, long reportStatAfterUpdatesNum) { + TtlVerifyUpdateFunction( + @Nonnull StateTtlConfig ttlConfig, + MonotonicTTLTimeProvider ttlTimeProvider, + long reportStatAfterUpdatesNum) { this.ttlConfig = ttlConfig; - this.ttl = ttlConfig.getTtl().toMilliseconds(); + this.ttlTimeProvider = checkNotNull(ttlTimeProvider); this.stat = new UpdateStat(reportStatAfterUpdatesNum); } @@ -91,17 +97,13 @@ public void flatMap(TtlStateUpdate updates, Collector out) throws Except } private TtlVerificationContext generateUpdateAndVerificationContext( - TtlStateUpdate updates, TtlStateVerifier verifier) throws Exception { + TtlStateUpdate updates, + TtlStateVerifier verifier) throws Exception { + List> prevUpdates = getPrevUpdates(verifier.getId()); Object update = updates.getUpdate(verifier.getId()); TtlUpdateContext updateContext = performUpdate(verifier, update); - boolean clashes = updateClashesWithPrevUpdates(updateContext.getUpdateWithTs(), prevUpdates); - if (clashes) { - resetState(verifier.getId()); - prevUpdates = Collections.emptyList(); - updateContext = performUpdate(verifier, update); - } - stat.update(clashes, prevUpdates.size()); + stat.update(prevUpdates.size()); prevUpdatesByVerifierId.get(verifier.getId()).add(updateContext.getUpdateWithTs()); return new TtlVerificationContext<>(updates.getKey(), verifier.getId(), prevUpdates, updateContext); } @@ -113,33 +115,22 @@ private List> getPrevUpdates(String verifierId) throws Exception } private TtlUpdateContext performUpdate( - TtlStateVerifier verifier, Object update) throws Exception { + TtlStateVerifier verifier, + Object update) throws Exception { + + final long timestampBeforeUpdate = ttlTimeProvider.currentTimestamp(); State state = states.get(verifier.getId()); - long timestampBeforeUpdate = System.currentTimeMillis(); Object valueBeforeUpdate = verifier.get(state); verifier.update(state, update); Object updatedValue = verifier.get(state); - return new TtlUpdateContext<>(timestampBeforeUpdate, - valueBeforeUpdate, update, updatedValue, System.currentTimeMillis()); - } + final long timestampAfterUpdate = ttlTimeProvider.unfreezeTime(); - private boolean updateClashesWithPrevUpdates(ValueWithTs update, List> prevUpdates) { - return tooSlow(update) || - (!prevUpdates.isEmpty() && prevUpdates.stream().anyMatch(pu -> updatesClash(pu, update))); - } - - private boolean tooSlow(ValueWithTs update) { - return update.getTimestampAfterUpdate() - update.getTimestampBeforeUpdate() >= ttl; - } + checkState( + timestampAfterUpdate == timestampBeforeUpdate, + "Timestamps before and after the update do not match." + ); - private boolean updatesClash(ValueWithTs prevUpdate, ValueWithTs nextUpdate) { - return prevUpdate.getTimestampAfterUpdate() + ttl >= nextUpdate.getTimestampBeforeUpdate() && - prevUpdate.getTimestampBeforeUpdate() + ttl <= nextUpdate.getTimestampAfterUpdate(); - } - - private void resetState(String verifierId) { - states.get(verifierId).clear(); - prevUpdatesByVerifierId.get(verifierId).clear(); + return new TtlUpdateContext<>(valueBeforeUpdate, update, updatedValue, timestampAfterUpdate); } @Override @@ -153,7 +144,7 @@ public void initializeState(FunctionInitializationContext context) { .collect(Collectors.toMap(TtlStateVerifier::getId, v -> v.createState(context, ttlConfig))); prevUpdatesByVerifierId = TtlStateVerifier.VERIFIERS.stream() .collect(Collectors.toMap(TtlStateVerifier::getId, v -> { - Preconditions.checkNotNull(v); + checkNotNull(v); TypeSerializer> typeSerializer = new ValueWithTs.Serializer(v.getUpdateSerializer()); ListStateDescriptor> stateDesc = new ListStateDescriptor<>( "TtlPrevValueState_" + v.getId(), typeSerializer); @@ -165,22 +156,17 @@ public void initializeState(FunctionInitializationContext context) { private static class UpdateStat implements Serializable { final long reportStatAfterUpdatesNum; long updates = 0; - long clashes = 0; long prevUpdatesNum = 0; UpdateStat(long reportStatAfterUpdatesNum) { this.reportStatAfterUpdatesNum = reportStatAfterUpdatesNum; } - void update(boolean clash, long prevUpdatesSize) { + void update(long prevUpdatesSize) { updates++; - if (clash) { - clashes++; - } prevUpdatesNum += prevUpdatesSize; if (updates % reportStatAfterUpdatesNum == 0) { - LOG.info(String.format("Avg update chain length: %d, clash stat: %d/%d", - prevUpdatesNum / updates, clashes, updates)); + LOG.info(String.format("Avg update chain length: %d", prevUpdatesNum / updates)); } } } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java index 7b6def24204ec..46becbbbc18c4 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/AbstractTtlStateVerifier.java @@ -86,15 +86,18 @@ public void update(@Nonnull State state, Object update) throws Exception { @Override public boolean verify(@Nonnull TtlVerificationContext verificationContextRaw) { TtlVerificationContext verificationContext = (TtlVerificationContext) verificationContextRaw; - List> updates = new ArrayList<>(verificationContext.getPrevUpdates()); - long currentTimestamp = verificationContext.getUpdateContext().getTimestampBeforeUpdate(); - GV prevValue = expected(updates, currentTimestamp); + long currentTimestamp = verificationContext.getUpdateContext().getTimestamp(); + GV valueBeforeUpdate = verificationContext.getUpdateContext().getValueBeforeUpdate(); + List> updates = new ArrayList<>(verificationContext.getPrevUpdates()); + GV expectedValueBeforeUpdate = expected(updates, currentTimestamp); + + GV valueAfterUpdate = verificationContext.getUpdateContext().getValueAfterUpdate(); ValueWithTs update = verificationContext.getUpdateContext().getUpdateWithTs(); - GV updatedValue = verificationContext.getUpdateContext().getUpdatedValue(); updates.add(update); - GV expectedValue = expected(updates, currentTimestamp); - return Objects.equals(valueBeforeUpdate, prevValue) && Objects.equals(updatedValue, expectedValue); + GV expectedValueAfterUpdate = expected(updates, currentTimestamp); + + return Objects.equals(valueBeforeUpdate, expectedValueBeforeUpdate) && Objects.equals(valueAfterUpdate, expectedValueAfterUpdate); } abstract GV expected(@Nonnull List> updates, long currentTimestamp); diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java index 960bbe72401ff..8a629578679bd 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlAggregatingStateVerifier.java @@ -71,13 +71,13 @@ String expected(@Nonnull List> updates, long currentTimesta return null; } long acc = AGG_FUNC.createAccumulator(); - long lastTs = updates.get(0).getTimestampAfterUpdate(); + long lastTs = updates.get(0).getTimestamp(); for (ValueWithTs update : updates) { - if (expired(lastTs, update.getTimestampAfterUpdate())) { + if (expired(lastTs, update.getTimestamp())) { acc = AGG_FUNC.createAccumulator(); } acc = AGG_FUNC.add(update.getValue(), acc); - lastTs = update.getTimestampAfterUpdate(); + lastTs = update.getTimestamp(); } return expired(lastTs, currentTimestamp) ? null : AGG_FUNC.getResult(acc); } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java index c1cc761b0f3f1..bcc8590586381 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlFoldingStateVerifier.java @@ -74,13 +74,13 @@ Long expected(@Nonnull List> updates, long currentTimestamp return null; } long acc = INIT_VAL; - long lastTs = updates.get(0).getTimestampAfterUpdate(); + long lastTs = updates.get(0).getTimestamp(); for (ValueWithTs update : updates) { - if (expired(lastTs, update.getTimestampAfterUpdate())) { + if (expired(lastTs, update.getTimestamp())) { acc = INIT_VAL; } acc += update.getValue(); - lastTs = update.getTimestampAfterUpdate(); + lastTs = update.getTimestamp(); } return expired(lastTs, currentTimestamp) ? null : acc; } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java index b355aa9861a48..4aed98ffd2fb0 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlListStateVerifier.java @@ -71,7 +71,7 @@ void updateInternal(@Nonnull ListState state, String update) throws Exce @Nonnull List expected(@Nonnull List> updates, long currentTimestamp) { return updates.stream() - .filter(u -> !expired(u.getTimestampAfterUpdate(), currentTimestamp)) + .filter(u -> !expired(u.getTimestamp(), currentTimestamp)) .map(ValueWithTs::getValue) .collect(Collectors.toList()); } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java index a9d6b36f62f08..eeda78d73c213 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlMapStateVerifier.java @@ -87,7 +87,7 @@ Map expected(@Nonnull List>> .collect(Collectors.groupingBy(u -> u.getValue().f0)) .entrySet().stream() .map(e -> e.getValue().get(e.getValue().size() - 1)) - .filter(u -> !expired(u.getTimestampAfterUpdate(), currentTimestamp)) + .filter(u -> !expired(u.getTimestamp(), currentTimestamp)) .map(ValueWithTs::getValue) .collect(Collectors.toMap(u -> u.f0, u -> u.f1)); } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java index 773be05e04b5d..cd33ed08f4eb4 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlReducingStateVerifier.java @@ -73,13 +73,13 @@ Integer expected(@Nonnull List> updates, long currentTimest return null; } int acc = 0; - long lastTs = updates.get(0).getTimestampAfterUpdate(); + long lastTs = updates.get(0).getTimestamp(); for (ValueWithTs update : updates) { - if (expired(lastTs, update.getTimestampAfterUpdate())) { + if (expired(lastTs, update.getTimestamp())) { acc = 0; } acc += update.getValue(); - lastTs = update.getTimestampAfterUpdate(); + lastTs = update.getTimestamp(); } return expired(lastTs, currentTimestamp) ? null : acc; } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java index 959340b408d9c..61bf9e22c6306 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlUpdateContext.java @@ -24,25 +24,25 @@ /** Contains context relevant for state update with TTL. */ public class TtlUpdateContext implements Serializable { - private final long timestampBeforeUpdate; + private final GV valueBeforeUpdate; private final UV update; - private final GV updatedValue; - private final long timestampAfterUpdate; + private final GV valueAfterUpdate; + private final long timestamp; public TtlUpdateContext( - long timestampBeforeUpdate, - GV valueBeforeUpdate, UV update, GV updatedValue, - long timestampAfterUpdate) { + GV valueBeforeUpdate, + UV update, + GV updatedValue, + long timestamp) { this.valueBeforeUpdate = valueBeforeUpdate; this.update = update; - this.updatedValue = updatedValue; - this.timestampBeforeUpdate = timestampBeforeUpdate; - this.timestampAfterUpdate = timestampAfterUpdate; + this.valueAfterUpdate = updatedValue; + this.timestamp = timestamp; } - long getTimestampBeforeUpdate() { - return timestampBeforeUpdate; + long getTimestamp() { + return timestamp; } GV getValueBeforeUpdate() { @@ -51,21 +51,20 @@ GV getValueBeforeUpdate() { @Nonnull public ValueWithTs getUpdateWithTs() { - return new ValueWithTs<>(update, timestampBeforeUpdate, timestampAfterUpdate); + return new ValueWithTs<>(update, timestamp); } - GV getUpdatedValue() { - return updatedValue; + GV getValueAfterUpdate() { + return valueAfterUpdate; } @Override public String toString() { return "TtlUpdateContext{" + - "timestampBeforeUpdate=" + timestampBeforeUpdate + - ", valueBeforeUpdate=" + valueBeforeUpdate + + "valueBeforeUpdate=" + valueBeforeUpdate + ", update=" + update + - ", updatedValue=" + updatedValue + - ", timestampAfterUpdate=" + timestampAfterUpdate + + ", valueAfterUpdate=" + valueAfterUpdate + + ", timestamp=" + timestamp + '}'; } } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java index fa4929b933f41..d8bdfd4e79de9 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlValueStateVerifier.java @@ -61,6 +61,6 @@ String expected(@Nonnull List> updates, long currentTimestam return null; } ValueWithTs lastUpdate = updates.get(updates.size() - 1); - return expired(lastUpdate.getTimestampAfterUpdate(), currentTimestamp) ? null : lastUpdate.getValue(); + return expired(lastUpdate.getTimestamp(), currentTimestamp) ? null : lastUpdate.getValue(); } } diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java index 4c985cd525b31..6d04c4cb21928 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/TtlVerificationContext.java @@ -36,10 +36,10 @@ public class TtlVerificationContext implements Serializable { @SuppressWarnings("unchecked") public TtlVerificationContext( - int key, - @Nonnull String verifierId, - @Nonnull List> prevUpdates, - @Nonnull TtlUpdateContext updateContext) { + int key, + @Nonnull String verifierId, + @Nonnull List> prevUpdates, + @Nonnull TtlUpdateContext updateContext) { this.key = key; this.verifierId = verifierId; this.prevUpdates = new ArrayList<>(); diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java index 9302377d9eddb..a4f30804e4fd2 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/verify/ValueWithTs.java @@ -30,33 +30,26 @@ /** User state value with timestamps before and after update. */ public class ValueWithTs implements Serializable { private final V value; - private final long timestampBeforeUpdate; - private final long timestampAfterUpdate; + private final long timestamp; - public ValueWithTs(V value, long timestampBeforeUpdate, long timestampAfterUpdate) { + public ValueWithTs(V value, long timestamp) { this.value = value; - this.timestampBeforeUpdate = timestampBeforeUpdate; - this.timestampAfterUpdate = timestampAfterUpdate; + this.timestamp = timestamp; } V getValue() { return value; } - public long getTimestampBeforeUpdate() { - return timestampBeforeUpdate; - } - - public long getTimestampAfterUpdate() { - return timestampAfterUpdate; + long getTimestamp() { + return timestamp; } @Override public String toString() { return "ValueWithTs{" + "value=" + value + - ", timestampBeforeUpdate=" + timestampBeforeUpdate + - ", timestampAfterUpdate=" + timestampAfterUpdate + + ", timestamp=" + timestamp + '}'; } @@ -64,7 +57,7 @@ public String toString() { public static class Serializer extends CompositeSerializer> { public Serializer(TypeSerializer userValueSerializer) { - super(true, userValueSerializer, LongSerializer.INSTANCE, LongSerializer.INSTANCE); + super(true, userValueSerializer, LongSerializer.INSTANCE); } @SuppressWarnings("unchecked") @@ -74,7 +67,7 @@ public Serializer(TypeSerializer userValueSerializer) { @Override public ValueWithTs createInstance(@Nonnull Object ... values) { - return new ValueWithTs<>(values[0], (Long) values[1], (Long) values[2]); + return new ValueWithTs<>(values[0], (Long) values[1]); } @Override @@ -88,9 +81,7 @@ protected Object getField(@Nonnull ValueWithTs value, int index) { case 0: return value.getValue(); case 1: - return value.getTimestampBeforeUpdate(); - case 2: - return value.getTimestampAfterUpdate(); + return value.getTimestamp(); default: throw new FlinkRuntimeException("Unexpected field index for ValueWithTs"); } @@ -99,8 +90,8 @@ protected Object getField(@Nonnull ValueWithTs value, int index) { @SuppressWarnings("unchecked") @Override protected CompositeSerializer> createSerializerInstance( - PrecomputedParameters precomputed, - TypeSerializer... originalSerializers) { + PrecomputedParameters precomputed, + TypeSerializer... originalSerializers) { return new Serializer(precomputed, (TypeSerializer) originalSerializers[0]); } } From 0cc5df464399ddae46536140e3b48fa2f64315c3 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 13 Nov 2018 17:44:17 +0100 Subject: [PATCH 098/359] [FLINK-10869] [build] Remove outdated secrets from travis build The secrets are replaced with build server environment variables. --- .travis.yml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index e2291f64a77a7..a7b1eebb4929d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -41,10 +41,6 @@ env: global: # Global variable to avoid hanging travis builds when downloading cache archives. - MALLOC_ARENA_MAX=2 - # Build artifacts like logs (variables for apache/flink repo) - - secure: "gL3QRn6/XyVK+Em9RmVqpM6nbTwlhjK4/JiRYZGGCkBgTq4ZnG+Eq2qKAO22TAsqRSi7g7WAoAhUulPt0SJqH7hjMe0LetbO0izbVXDefwf2PJlsNgBbuFG6604++VUaUEyfPYYw9ADjV59LWG7+B/fjbRsevqRBZ30b1gv/tQQ=" - - secure: "eM9r8IglvnUKctxz/ga6hwGnCpdOvGyYdGj0H/UiNDEx3Lq1A6yp3gChEIXGJqRUXDI5TaIuidunUGY7KHml8urm8eG2Yk2ttxXehZqLpEaOU2jdNJCdLX8tlVfh14T9bxG5AYHQEV3qJUqDFtfXD3whvzuinrm1oEIA3qUxiA8=" - - secure: "EQYDWgJM5ANJ/sAFwmSEwSTOe9CDN/ENyQAr5/ntM67XanhTZj2Amgt9LthCRUU4EEPl/OFUTwNHMpv/+wa3q7dwVFldSIg5wyCndzJSATPyPBVjYgsXIQZVIjsq4TwTyrTteT55V6Oz2+l27Fvung2FPuN83ovswsJePFzMBxI=" - DOCKER_COMPOSE_VERSION=1.22.0 before_script: From 68b4545d932d405b5e466327fcda4728973281c8 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 13:34:13 +0100 Subject: [PATCH 099/359] [FLINK-10869] [tests] Add S3Credentials to get access to S3 test credentials --- .../flink/testutils/s3/S3Credentials.java | 106 ++++++++++++++++ .../flink/testutils/s3/S3TestCredentials.java | 117 ++++++++++++++++++ 2 files changed, 223 insertions(+) create mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java create mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java new file mode 100644 index 0000000000000..8f13ffaf92f79 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java @@ -0,0 +1,106 @@ +/* + * 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.testutils.s3; + +import org.junit.Assume; +import org.junit.AssumptionViolatedException; + +import javax.annotation.Nullable; + +/** + * Access to credentials to access S3 buckets during integration tests. + */ +public class S3Credentials { + + @Nullable + private static final String S3_TEST_BUCKET = System.getenv("IT_CASE_S3_BUCKET"); + + @Nullable + private static final String S3_TEST_ACCESS_KEY = System.getenv("IT_CASE_S3_ACCESS_KEY"); + + @Nullable + private static final String S3_TEST_SECRET_KEY = System.getenv("IT_CASE_S3_SECRET_KEY"); + + // ------------------------------------------------------------------------ + + /** + * Checks whether S3 test credentials are available in the environment variables + * of this JVM. + */ + public static boolean credentialsAvailable() { + return S3_TEST_BUCKET != null && S3_TEST_ACCESS_KEY != null && S3_TEST_SECRET_KEY != null; + } + + /** + * Checks whether credentials are available in the environment variables of this JVM. + * If not, throws an {@link AssumptionViolatedException} which causes JUnit tests to be + * skipped. + */ + public static void assumeCredentialsAvailable() { + Assume.assumeTrue("No S3 credentials available in this test's environment", credentialsAvailable()); + } + + /** + * Gets the S3 Access Key. + * + *

This method throws an exception if the key is not available. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public String getS3AccessKey() { + if (S3_TEST_ACCESS_KEY != null) { + return S3_TEST_ACCESS_KEY; + } + else { + throw new IllegalStateException("S3 test access key not available"); + } + } + + /** + * Gets the S3 Secret Key. + * + *

This method throws an exception if the key is not available. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public String getS3SecretKey() { + if (S3_TEST_SECRET_KEY != null) { + return S3_TEST_SECRET_KEY; + } + else { + throw new IllegalStateException("S3 test secret key not available"); + } + } + + /** + * Gets the URI for the path under which all tests should put their data. + * + *

This method throws an exception if the bucket was not configured. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public String getBaseUri() { + if (S3_TEST_BUCKET != null) { + return "s3://" + S3_TEST_BUCKET + "/temp/"; + } + else { + throw new IllegalStateException("S3 test bucket not available"); + } + } +} diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java new file mode 100644 index 0000000000000..b1a0aecf07971 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.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.testutils.s3; + +import org.junit.Assume; +import org.junit.AssumptionViolatedException; + +import javax.annotation.Nullable; + +/** + * Access to credentials to access S3 buckets during integration tests. + */ +public class S3TestCredentials { + + @Nullable + private static final String S3_TEST_BUCKET = System.getenv("IT_CASE_S3_BUCKET"); + + @Nullable + private static final String S3_TEST_ACCESS_KEY = System.getenv("IT_CASE_S3_ACCESS_KEY"); + + @Nullable + private static final String S3_TEST_SECRET_KEY = System.getenv("IT_CASE_S3_SECRET_KEY"); + + // ------------------------------------------------------------------------ + + /** + * Checks whether S3 test credentials are available in the environment variables + * of this JVM. + */ + public static boolean credentialsAvailable() { + return S3_TEST_BUCKET != null && S3_TEST_ACCESS_KEY != null && S3_TEST_SECRET_KEY != null; + } + + /** + * Checks whether credentials are available in the environment variables of this JVM. + * If not, throws an {@link AssumptionViolatedException} which causes JUnit tests to be + * skipped. + */ + public static void assumeCredentialsAvailable() { + Assume.assumeTrue("No S3 credentials available in this test's environment", credentialsAvailable()); + } + + /** + * Gets the S3 Access Key. + * + *

This method throws an exception if the key is not available. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public static String getS3AccessKey() { + if (S3_TEST_ACCESS_KEY != null) { + return S3_TEST_ACCESS_KEY; + } + else { + throw new IllegalStateException("S3 test access key not available"); + } + } + + /** + * Gets the S3 Secret Key. + * + *

This method throws an exception if the key is not available. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public static String getS3SecretKey() { + if (S3_TEST_SECRET_KEY != null) { + return S3_TEST_SECRET_KEY; + } + else { + throw new IllegalStateException("S3 test secret key not available"); + } + } + + /** + * Gets the URI for the path under which all tests should put their data. + * + *

This method throws an exception if the bucket was not configured. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public static String getTestBucketUri() { + return getTestBucketUriWithScheme("s3"); + } + + /** + * Gets the URI for the path under which all tests should put their data. + * + *

This method throws an exception if the bucket was not configured. Tests should + * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not + * available. + */ + public static String getTestBucketUriWithScheme(String scheme) { + if (S3_TEST_BUCKET != null) { + return scheme + "://" + S3_TEST_BUCKET + "/temp/"; + } + else { + throw new IllegalStateException("S3 test bucket not available"); + } + } +} From fe358d608a6a7c925b9918d1ac3adeef24454cef Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 16:36:23 +0100 Subject: [PATCH 100/359] [FLINK-10869] [tests] Update all S3 tests to use new test credentials --- .../HadoopS3FileSystemBehaviorITCase.java | 17 +-- .../fs/s3hadoop/HadoopS3FileSystemITCase.java | 44 ++++---- ...adoopS3RecoverableWriterExceptionTest.java | 20 ++-- .../HadoopS3RecoverableWriterTest.java | 20 ++-- .../PrestoS3FileSystemBehaviorITCase.java | 17 +-- .../fs/s3presto/PrestoS3FileSystemITCase.java | 41 +++---- .../PrestoS3RecoverableWriterTest.java | 21 ++-- .../flink/testutils/s3/S3Credentials.java | 106 ------------------ .../flink/yarn/YarnFileStageTestS3ITCase.java | 25 ++--- 9 files changed, 76 insertions(+), 235 deletions(-) delete mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemBehaviorITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemBehaviorITCase.java index c8aaaeef74d05..df73cdae34cb1 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemBehaviorITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemBehaviorITCase.java @@ -23,9 +23,9 @@ import org.apache.flink.core.fs.FileSystemBehaviorTestSuite; import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.junit.AfterClass; -import org.junit.Assume; import org.junit.BeforeClass; import java.io.IOException; @@ -36,24 +36,17 @@ */ public class HadoopS3FileSystemBehaviorITCase extends FileSystemBehaviorTestSuite { - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); } @@ -69,7 +62,7 @@ public FileSystem getFileSystem() throws Exception { @Override public Path getBasePath() throws Exception { - return new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + return new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); } @Override diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java index 6dbdac511f4ac..2195bd0e7632c 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java @@ -25,10 +25,10 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.core.fs.Path; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; -import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -69,10 +69,6 @@ public static List parameters() { private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - /** * Will be updated by {@link #checkCredentialsAndSetup()} if the test is not skipped. */ @@ -81,18 +77,16 @@ public static List parameters() { @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); // check for uniqueness of the test directory - final Path directory = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + final Path directory = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); final FileSystem fs = directory.getFileSystem(); // directory must not yet exist @@ -108,11 +102,11 @@ public static void cleanUp() throws IOException, InterruptedException { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); - final Path directory = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + final Path directory = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); final FileSystem fs = directory.getFileSystem(); // clean up @@ -128,7 +122,7 @@ public static void cleanUp() throws IOException, InterruptedException { } private String getBasePath() { - return scheme + "://" + BUCKET + '/' + TEST_DATA_DIR + "/" + scheme; + return S3TestCredentials.getTestBucketUriWithScheme(scheme) + TEST_DATA_DIR + '/' + scheme; } @Test @@ -138,8 +132,8 @@ public void testConfigKeysForwarding() throws Exception { // standard Hadoop-style credential keys { Configuration conf = new Configuration(); - conf.setString("fs.s3a.access.key", ACCESS_KEY); - conf.setString("fs.s3a.secret.key", SECRET_KEY); + conf.setString("fs.s3a.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("fs.s3a.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem(); @@ -148,8 +142,8 @@ public void testConfigKeysForwarding() throws Exception { // shortened Hadoop-style credential keys { Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem(); @@ -158,8 +152,8 @@ public void testConfigKeysForwarding() throws Exception { // shortened Presto-style credential keys { Configuration conf = new Configuration(); - conf.setString("s3.access-key", ACCESS_KEY); - conf.setString("s3.secret-key", SECRET_KEY); + conf.setString("s3.access-key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret-key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem(); @@ -170,8 +164,8 @@ public void testConfigKeysForwarding() throws Exception { public void testSimpleFileWriteAndRead() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); final String testLine = "Hello Upload!"; @@ -208,8 +202,8 @@ public void testSimpleFileWriteAndRead() throws Exception { public void testDirectoryListing() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java index 634fa00344d84..3fb67c1414f5a 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java @@ -25,12 +25,12 @@ import org.apache.flink.core.fs.RecoverableFsDataOutputStream; import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.fs.s3.common.FlinkS3FileSystem; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.apache.flink.util.StringUtils; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.AfterClass; -import org.junit.Assume; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -55,10 +55,6 @@ public class HadoopS3RecoverableWriterExceptionTest extends TestLogger { // ----------------------- S3 general configuration ----------------------- - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final long PART_UPLOAD_MIN_SIZE_VALUE = 7L << 20; private static final int MAX_CONCURRENT_UPLOADS_VALUE = 2; @@ -66,9 +62,7 @@ public class HadoopS3RecoverableWriterExceptionTest extends TestLogger { private static final Random RND = new Random(); - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - - private static final Path basePath = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + private static Path basePath; private static FlinkS3FileSystem fileSystem; @@ -89,14 +83,14 @@ public class HadoopS3RecoverableWriterExceptionTest extends TestLogger { @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); + + basePath = new Path(S3TestCredentials.getTestBucketUri() + "tests-" + UUID.randomUUID()); // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); conf.setLong(PART_UPLOAD_MIN_SIZE, PART_UPLOAD_MIN_SIZE_VALUE); conf.setInteger(MAX_CONCURRENT_UPLOADS, MAX_CONCURRENT_UPLOADS_VALUE); diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java index 4a1368a815e86..a9027a0b62c8e 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java @@ -27,6 +27,7 @@ import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.fs.s3.common.FlinkS3FileSystem; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.apache.flink.util.MathUtils; import org.apache.flink.util.StringUtils; import org.apache.flink.util.TestLogger; @@ -34,7 +35,6 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; -import org.junit.Assume; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -63,10 +63,6 @@ public class HadoopS3RecoverableWriterTest extends TestLogger { // ----------------------- S3 general configuration ----------------------- - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final long PART_UPLOAD_MIN_SIZE_VALUE = 7L << 20; private static final int MAX_CONCURRENT_UPLOADS_VALUE = 2; @@ -74,9 +70,7 @@ public class HadoopS3RecoverableWriterTest extends TestLogger { private static final Random RND = new Random(); - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - - private static final Path basePath = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + private static Path basePath; private static FlinkS3FileSystem fileSystem; @@ -101,14 +95,14 @@ public class HadoopS3RecoverableWriterTest extends TestLogger { @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); + + basePath = new Path(S3TestCredentials.getTestBucketUri() + "tests-" + UUID.randomUUID()); // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); conf.setLong(PART_UPLOAD_MIN_SIZE, PART_UPLOAD_MIN_SIZE_VALUE); conf.setInteger(MAX_CONCURRENT_UPLOADS, MAX_CONCURRENT_UPLOADS_VALUE); diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemBehaviorITCase.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemBehaviorITCase.java index 812404ce63911..1603927749ef2 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemBehaviorITCase.java +++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemBehaviorITCase.java @@ -23,9 +23,9 @@ import org.apache.flink.core.fs.FileSystemBehaviorTestSuite; import org.apache.flink.core.fs.FileSystemKind; import org.apache.flink.core.fs.Path; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.junit.AfterClass; -import org.junit.Assume; import org.junit.BeforeClass; import java.io.IOException; @@ -36,24 +36,17 @@ */ public class PrestoS3FileSystemBehaviorITCase extends FileSystemBehaviorTestSuite { - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); } @@ -69,7 +62,7 @@ public FileSystem getFileSystem() throws Exception { @Override public Path getBasePath() throws Exception { - return new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); + return new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); } @Override diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java index cc5c9935202a6..9a008a176d66a 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemITCase.java @@ -25,9 +25,9 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem.WriteMode; import org.apache.flink.core.fs.Path; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.apache.flink.util.TestLogger; -import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -68,23 +68,16 @@ public static List parameters() { return Arrays.asList("s3", "s3p"); } - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - @BeforeClass public static void checkIfCredentialsArePresent() { - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); } @Test public void testConfigKeysForwarding() throws Exception { - final Path path = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR); + final Path path = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); // access without credentials should fail { @@ -103,8 +96,8 @@ public void testConfigKeysForwarding() throws Exception { { Configuration conf = new Configuration(); conf.setString(S3_USE_INSTANCE_CREDENTIALS, "false"); - conf.setString("presto.s3.access-key", ACCESS_KEY); - conf.setString("presto.s3.secret-key", SECRET_KEY); + conf.setString("presto.s3.access-key", S3TestCredentials.getS3AccessKey()); + conf.setString("presto.s3.secret-key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem().exists(path); @@ -114,8 +107,8 @@ public void testConfigKeysForwarding() throws Exception { { Configuration conf = new Configuration(); conf.setString(S3_USE_INSTANCE_CREDENTIALS, "false"); - conf.setString("s3.access-key", ACCESS_KEY); - conf.setString("s3.secret-key", SECRET_KEY); + conf.setString("s3.access-key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret-key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem().exists(path); @@ -125,8 +118,8 @@ public void testConfigKeysForwarding() throws Exception { { Configuration conf = new Configuration(); conf.setString(S3_USE_INSTANCE_CREDENTIALS, "false"); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem().exists(path); @@ -136,8 +129,8 @@ public void testConfigKeysForwarding() throws Exception { { Configuration conf = new Configuration(); conf.setString(S3_USE_INSTANCE_CREDENTIALS, "false"); - conf.setString("presto.s3.access.key", ACCESS_KEY); - conf.setString("presto.s3.secret.key", SECRET_KEY); + conf.setString("presto.s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("presto.s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); path.getFileSystem().exists(path); @@ -151,14 +144,14 @@ public void testConfigKeysForwarding() throws Exception { public void testSimpleFileWriteAndRead() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs final Configuration conf = new Configuration(); - conf.setString("s3.access-key", ACCESS_KEY); - conf.setString("s3.secret-key", SECRET_KEY); + conf.setString("s3.access-key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret-key", S3TestCredentials.getS3SecretKey()); final String testLine = "Hello Upload!"; FileSystem.initialize(conf); - final Path path = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR + "/test.txt"); + final Path path = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR + "/test.txt"); final FileSystem fs = path.getFileSystem(); try { @@ -186,12 +179,12 @@ public void testSimpleFileWriteAndRead() throws Exception { public void testDirectoryListing() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs final Configuration conf = new Configuration(); - conf.setString("s3.access-key", ACCESS_KEY); - conf.setString("s3.secret-key", SECRET_KEY); + conf.setString("s3.access-key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret-key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); - final Path directory = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR + "/testdir/"); + final Path directory = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR + "/testdir/"); final FileSystem fs = directory.getFileSystem(); // directory must not yet exist diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3RecoverableWriterTest.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3RecoverableWriterTest.java index 580d957db2327..59d2f165e2e14 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3RecoverableWriterTest.java +++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3RecoverableWriterTest.java @@ -21,15 +21,15 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; import org.apache.flink.fs.s3.common.FlinkS3FileSystem; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.junit.AfterClass; -import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.net.URI; import java.util.UUID; import static org.apache.flink.fs.s3.common.AbstractS3FileSystemFactory.MAX_CONCURRENT_UPLOADS; @@ -42,10 +42,6 @@ public class PrestoS3RecoverableWriterTest { // ----------------------- S3 general configuration ----------------------- - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final long PART_UPLOAD_MIN_SIZE_VALUE = 7L << 20; private static final int MAX_CONCURRENT_UPLOADS_VALUE = 2; @@ -53,21 +49,17 @@ public class PrestoS3RecoverableWriterTest { private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - private static final Path basePath = new Path("s3://" + BUCKET + '/' + TEST_DATA_DIR); - // ----------------------- Test Lifecycle ----------------------- @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); // initialize configuration with valid credentials final Configuration conf = new Configuration(); - conf.setString("s3.access.key", ACCESS_KEY); - conf.setString("s3.secret.key", SECRET_KEY); + conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); + conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); conf.setLong(PART_UPLOAD_MIN_SIZE, PART_UPLOAD_MIN_SIZE_VALUE); conf.setInteger(MAX_CONCURRENT_UPLOADS, MAX_CONCURRENT_UPLOADS_VALUE); @@ -87,7 +79,8 @@ public static void cleanUp() throws IOException { @Test(expected = UnsupportedOperationException.class) public void requestingRecoverableWriterShouldThroughException() throws Exception { - FlinkS3FileSystem fileSystem = (FlinkS3FileSystem) FileSystem.get(basePath.toUri()); + URI s3Uri = URI.create(S3TestCredentials.getTestBucketUri()); + FlinkS3FileSystem fileSystem = (FlinkS3FileSystem) FileSystem.get(s3Uri); fileSystem.createRecoverableWriter(); } } diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java deleted file mode 100644 index 8f13ffaf92f79..0000000000000 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3Credentials.java +++ /dev/null @@ -1,106 +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.testutils.s3; - -import org.junit.Assume; -import org.junit.AssumptionViolatedException; - -import javax.annotation.Nullable; - -/** - * Access to credentials to access S3 buckets during integration tests. - */ -public class S3Credentials { - - @Nullable - private static final String S3_TEST_BUCKET = System.getenv("IT_CASE_S3_BUCKET"); - - @Nullable - private static final String S3_TEST_ACCESS_KEY = System.getenv("IT_CASE_S3_ACCESS_KEY"); - - @Nullable - private static final String S3_TEST_SECRET_KEY = System.getenv("IT_CASE_S3_SECRET_KEY"); - - // ------------------------------------------------------------------------ - - /** - * Checks whether S3 test credentials are available in the environment variables - * of this JVM. - */ - public static boolean credentialsAvailable() { - return S3_TEST_BUCKET != null && S3_TEST_ACCESS_KEY != null && S3_TEST_SECRET_KEY != null; - } - - /** - * Checks whether credentials are available in the environment variables of this JVM. - * If not, throws an {@link AssumptionViolatedException} which causes JUnit tests to be - * skipped. - */ - public static void assumeCredentialsAvailable() { - Assume.assumeTrue("No S3 credentials available in this test's environment", credentialsAvailable()); - } - - /** - * Gets the S3 Access Key. - * - *

This method throws an exception if the key is not available. Tests should - * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not - * available. - */ - public String getS3AccessKey() { - if (S3_TEST_ACCESS_KEY != null) { - return S3_TEST_ACCESS_KEY; - } - else { - throw new IllegalStateException("S3 test access key not available"); - } - } - - /** - * Gets the S3 Secret Key. - * - *

This method throws an exception if the key is not available. Tests should - * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not - * available. - */ - public String getS3SecretKey() { - if (S3_TEST_SECRET_KEY != null) { - return S3_TEST_SECRET_KEY; - } - else { - throw new IllegalStateException("S3 test secret key not available"); - } - } - - /** - * Gets the URI for the path under which all tests should put their data. - * - *

This method throws an exception if the bucket was not configured. Tests should - * use {@link #assumeCredentialsAvailable()} to skip tests when credentials are not - * available. - */ - public String getBaseUri() { - if (S3_TEST_BUCKET != null) { - return "s3://" + S3_TEST_BUCKET + "/temp/"; - } - else { - throw new IllegalStateException("S3 test bucket not available"); - } - } -} diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java index e1e95b1c379d1..bd9d6996babeb 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java @@ -23,10 +23,10 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; +import org.apache.flink.testutils.s3.S3TestCredentials; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; -import org.junit.Assume; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Rule; @@ -53,13 +53,8 @@ */ public class YarnFileStageTestS3ITCase extends TestLogger { - private static final String BUCKET = System.getenv("ARTIFACTS_AWS_BUCKET"); - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - private static final String ACCESS_KEY = System.getenv("ARTIFACTS_AWS_ACCESS_KEY"); - private static final String SECRET_KEY = System.getenv("ARTIFACTS_AWS_SECRET_KEY"); - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); @@ -79,9 +74,7 @@ public class YarnFileStageTestS3ITCase extends TestLogger { @BeforeClass public static void checkCredentialsAndSetup() throws IOException { // check whether credentials exist - Assume.assumeTrue("AWS S3 bucket not configured, skipping test...", BUCKET != null); - Assume.assumeTrue("AWS S3 access key not configured, skipping test...", ACCESS_KEY != null); - Assume.assumeTrue("AWS S3 secret key not configured, skipping test...", SECRET_KEY != null); + S3TestCredentials.assumeCredentialsAvailable(); skipTest = false; @@ -115,14 +108,14 @@ private static void setupCustomHadoopConfig() throws IOException { Map parameters = new HashMap<>(); // set all different S3 fs implementation variants' configuration keys - parameters.put("fs.s3a.access.key", ACCESS_KEY); - parameters.put("fs.s3a.secret.key", SECRET_KEY); + parameters.put("fs.s3a.access.key", S3TestCredentials.getS3AccessKey()); + parameters.put("fs.s3a.secret.key", S3TestCredentials.getS3SecretKey()); - parameters.put("fs.s3.awsAccessKeyId", ACCESS_KEY); - parameters.put("fs.s3.awsSecretAccessKey", SECRET_KEY); + parameters.put("fs.s3.awsAccessKeyId", S3TestCredentials.getS3AccessKey()); + parameters.put("fs.s3.awsSecretAccessKey", S3TestCredentials.getS3SecretKey()); - parameters.put("fs.s3n.awsAccessKeyId", ACCESS_KEY); - parameters.put("fs.s3n.awsSecretAccessKey", SECRET_KEY); + parameters.put("fs.s3n.awsAccessKeyId", S3TestCredentials.getS3AccessKey()); + parameters.put("fs.s3n.awsSecretAccessKey", S3TestCredentials.getS3SecretKey()); try (PrintStream out = new PrintStream(new FileOutputStream(hadoopConfig))) { out.println(""); @@ -155,7 +148,7 @@ private static void setupCustomHadoopConfig() throws IOException { private void testRecursiveUploadForYarn(String scheme, String pathSuffix) throws Exception { ++numRecursiveUploadTests; - final Path basePath = new Path(scheme + "://" + BUCKET + '/' + TEST_DATA_DIR); + final Path basePath = new Path(S3TestCredentials.getTestBucketUriWithScheme(scheme) + TEST_DATA_DIR); final HadoopFileSystem fs = (HadoopFileSystem) basePath.getFileSystem(); assumeFalse(fs.exists(basePath)); From bbbb9aadc3405635dc385e025101dcf268f39ea4 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 22:52:36 +0100 Subject: [PATCH 101/359] [FLINK-10869] [tests] Remove Yarn staging test for deprecated Hadoop s3:// filesystem. That filesystem is no longer supported by Hadoop, is discouraged to use, and is unable to work with any non-overall-permissive permission setup. --- .../flink/yarn/YarnFileStageTestS3ITCase.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java index bd9d6996babeb..0fa9893c05438 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java @@ -164,23 +164,6 @@ private void testRecursiveUploadForYarn(String scheme, String pathSuffix) throws } } - /** - * Verifies that nested directories are properly copied with a s3a:// file - * systems during resource uploads for YARN. - */ - @Test - public void testRecursiveUploadForYarnS3() throws Exception { - try { - Class.forName("org.apache.hadoop.fs.s3.S3FileSystem"); - } catch (ClassNotFoundException e) { - // not in the classpath, cannot run this test - String msg = "Skipping test because S3FileSystem is not in the class path"; - log.info(msg); - assumeNoException(msg, e); - } - testRecursiveUploadForYarn("s3", "testYarn-s3"); - } - @Test public void testRecursiveUploadForYarnS3n() throws Exception { try { From d3bd21a51bc3811eb898eebcb7c7ba61c8682aee Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 23:23:30 +0100 Subject: [PATCH 102/359] [FLINK-10869] [tests] Adjust end-2-end tests to new S3 credentials --- .../test-scripts/common_s3.sh | 60 +++++++++---------- .../test-scripts/test_shaded_hadoop_s3a.sh | 4 +- .../test-scripts/test_shaded_presto_s3.sh | 4 +- .../test-scripts/test_streaming_file_sink.sh | 2 +- 4 files changed, 35 insertions(+), 35 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3.sh b/flink-end-to-end-tests/test-scripts/common_s3.sh index 5c16bb75bea43..784d5739eacee 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3.sh @@ -17,30 +17,30 @@ # limitations under the License. ################################################################################ -if [[ -z "$ARTIFACTS_AWS_BUCKET" ]]; then +if [[ -z "$IT_CASE_S3_BUCKET" ]]; then echo "Did not find AWS environment variables, NOT running the e2e test." exit 0 else - echo "Found AWS bucket $ARTIFACTS_AWS_BUCKET, running the e2e test." + echo "Found AWS bucket $IT_CASE_S3_BUCKET, running the e2e test." fi -if [[ -z "$ARTIFACTS_AWS_ACCESS_KEY" ]]; then +if [[ -z "$IT_CASE_S3_ACCESS_KEY" ]]; then echo "Did not find AWS environment variables, NOT running the e2e test." exit 0 else - echo "Found AWS access key $ARTIFACTS_AWS_ACCESS_KEY, running the e2e test." + echo "Found AWS access key $IT_CASE_S3_ACCESS_KEY, running the e2e test." fi -if [[ -z "$ARTIFACTS_AWS_SECRET_KEY" ]]; then +if [[ -z "$IT_CASE_S3_SECRET_KEY" ]]; then echo "Did not find AWS environment variables, NOT running the e2e test." exit 0 else - echo "Found AWS secret key $ARTIFACTS_AWS_SECRET_KEY, running the e2e test." + echo "Found AWS secret key $IT_CASE_S3_SECRET_KEY, running the e2e test." fi -AWS_REGION="${AWS_REGION:-eu-west-1}" -AWS_ACCESS_KEY=$ARTIFACTS_AWS_ACCESS_KEY -AWS_SECRET_KEY=$ARTIFACTS_AWS_SECRET_KEY +AWS_REGION="${AWS_REGION:-us-east-1}" +AWS_ACCESS_KEY=$IT_CASE_S3_ACCESS_KEY +AWS_SECRET_KEY=$IT_CASE_S3_SECRET_KEY s3util="java -jar ${END_TO_END_DIR}/flink-e2e-test-utils/target/S3UtilProgram.jar" @@ -49,8 +49,8 @@ s3util="java -jar ${END_TO_END_DIR}/flink-e2e-test-utils/target/S3UtilProgram.ja # # Globals: # FLINK_DIR -# ARTIFACTS_AWS_ACCESS_KEY -# ARTIFACTS_AWS_SECRET_KEY +# IT_CASE_S3_ACCESS_KEY +# IT_CASE_S3_SECRET_KEY # Arguments: # None # Returns: @@ -68,8 +68,8 @@ function s3_setup { trap s3_cleanup EXIT cp $FLINK_DIR/opt/flink-s3-fs-hadoop-*.jar $FLINK_DIR/lib/ - echo "s3.access-key: $ARTIFACTS_AWS_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" - echo "s3.secret-key: $ARTIFACTS_AWS_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" + echo "s3.access-key: $IT_CASE_S3_ACCESS_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" + echo "s3.secret-key: $IT_CASE_S3_SECRET_KEY" >> "$FLINK_DIR/conf/flink-conf.yaml" } s3_setup @@ -78,7 +78,7 @@ s3_setup # List s3 objects by full path prefix. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - s3 full path key prefix # Returns: @@ -86,14 +86,14 @@ s3_setup ################################### function s3_list { AWS_REGION=$AWS_REGION \ - ${s3util} --action listByFullPathPrefix --s3prefix "$1" --bucket $ARTIFACTS_AWS_BUCKET + ${s3util} --action listByFullPathPrefix --s3prefix "$1" --bucket $IT_CASE_S3_BUCKET } ################################### # Download s3 object. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - local path to save file # $2 - s3 object key @@ -102,14 +102,14 @@ function s3_list { ################################### function s3_get { AWS_REGION=$AWS_REGION \ - ${s3util} --action downloadFile --localFile "$1" --s3file "$2" --bucket $ARTIFACTS_AWS_BUCKET + ${s3util} --action downloadFile --localFile "$1" --s3file "$2" --bucket $IT_CASE_S3_BUCKET } ################################### # Download s3 objects to folder by full path prefix. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - local path to save folder with files # $2 - s3 key full path prefix @@ -121,14 +121,14 @@ function s3_get_by_full_path_and_filename_prefix { local file_prefix="${3-}" AWS_REGION=$AWS_REGION \ ${s3util} --action downloadByFullPathAndFileNamePrefix \ - --localFolder "$1" --s3prefix "$2" --s3filePrefix "${file_prefix}" --bucket $ARTIFACTS_AWS_BUCKET + --localFolder "$1" --s3prefix "$2" --s3filePrefix "${file_prefix}" --bucket $IT_CASE_S3_BUCKET } ################################### # Upload file to s3 object. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - local file to upload # $2 - s3 bucket @@ -144,8 +144,8 @@ function s3_put { contentType="application/octet-stream" dateValue=`date -R` stringToSign="PUT\n\n${contentType}\n${dateValue}\n${resource}" - s3Key=$ARTIFACTS_AWS_ACCESS_KEY - s3Secret=$ARTIFACTS_AWS_SECRET_KEY + s3Key=$IT_CASE_S3_ACCESS_KEY + s3Secret=$IT_CASE_S3_SECRET_KEY signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` curl -X PUT -T "${local_file}" \ -H "Host: ${bucket}.s3.amazonaws.com" \ @@ -174,8 +174,8 @@ function s3_delete { contentType="application/octet-stream" dateValue=`date -R` stringToSign="DELETE\n\n${contentType}\n${dateValue}\n${resource}" - s3Key=$ARTIFACTS_AWS_ACCESS_KEY - s3Secret=$ARTIFACTS_AWS_SECRET_KEY + s3Key=$IT_CASE_S3_ACCESS_KEY + s3Secret=$IT_CASE_S3_SECRET_KEY signature=`echo -en ${stringToSign} | openssl sha1 -hmac ${s3Secret} -binary | base64` curl -X DELETE \ -H "Host: ${bucket}.s3.amazonaws.com" \ @@ -189,7 +189,7 @@ function s3_delete { # Delete s3 objects by full path prefix. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - s3 key full path prefix # Returns: @@ -197,7 +197,7 @@ function s3_delete { ################################### function s3_delete_by_full_path_prefix { AWS_REGION=$AWS_REGION \ - ${s3util} --action deleteByFullPathPrefix --s3prefix "$1" --bucket $ARTIFACTS_AWS_BUCKET + ${s3util} --action deleteByFullPathPrefix --s3prefix "$1" --bucket $IT_CASE_S3_BUCKET } ################################### @@ -206,7 +206,7 @@ function s3_delete_by_full_path_prefix { # because SQL is used to query the s3 object. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - s3 file object key # $2 - s3 bucket @@ -215,7 +215,7 @@ function s3_delete_by_full_path_prefix { ################################### function s3_get_number_of_lines_in_file { AWS_REGION=$AWS_REGION \ - ${s3util} --action numberOfLinesInFile --s3file "$1" --bucket $ARTIFACTS_AWS_BUCKET + ${s3util} --action numberOfLinesInFile --s3file "$1" --bucket $IT_CASE_S3_BUCKET } ################################### @@ -224,7 +224,7 @@ function s3_get_number_of_lines_in_file { # because SQL is used to query the s3 objects. # # Globals: -# ARTIFACTS_AWS_BUCKET +# IT_CASE_S3_BUCKET # Arguments: # $1 - s3 key prefix # $2 - s3 bucket @@ -236,5 +236,5 @@ function s3_get_number_of_lines_by_prefix { local file_prefix="${3-}" AWS_REGION=$AWS_REGION \ ${s3util} --action numberOfLinesInFilesWithFullAndNamePrefix \ - --s3prefix "$1" --s3filePrefix "${file_prefix}" --bucket $ARTIFACTS_AWS_BUCKET + --s3prefix "$1" --s3filePrefix "${file_prefix}" --bucket $IT_CASE_S3_BUCKET } diff --git a/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh index 3d8386758526a..e6ae9929aef23 100755 --- a/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh +++ b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh @@ -22,10 +22,10 @@ source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/common_s3.sh -s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a +s3_put $TEST_INFRA_DIR/test-data/words $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-s3a # make sure we delete the file at the end function shaded_s3a_cleanup { - s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-s3a + s3_delete $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-s3a } trap shaded_s3a_cleanup EXIT diff --git a/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh index bd33b410dfd4a..0421c840a3dc6 100755 --- a/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh +++ b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh @@ -22,10 +22,10 @@ source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/common_s3.sh -s3_put $TEST_INFRA_DIR/test-data/words $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 +s3_put $TEST_INFRA_DIR/test-data/words $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-presto-s3 # make sure we delete the file at the end function shaded_presto_s3_cleanup { - s3_delete $ARTIFACTS_AWS_BUCKET flink-end-to-end-test-shaded-presto-s3 + s3_delete $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-presto-s3 } trap shaded_presto_s3_cleanup EXIT diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh index 6c8d0b8543566..7a469aa7951ae 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh @@ -26,7 +26,7 @@ set_conf_ssl "mutual" OUT=out OUTPUT_PATH="$TEST_DATA_DIR/$OUT" -S3_OUTPUT_PATH="s3://$ARTIFACTS_AWS_BUCKET/$OUT" +S3_OUTPUT_PATH="s3://$IT_CASE_S3_BUCKET/temp/$OUT" mkdir -p $OUTPUT_PATH From ba66aedad70b152d129528d11271c89575161a97 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 23:34:28 +0100 Subject: [PATCH 103/359] [hotfix] [e2e] Do not print S3 credentials in test scripts --- flink-end-to-end-tests/test-scripts/common_s3.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3.sh b/flink-end-to-end-tests/test-scripts/common_s3.sh index 784d5739eacee..1f3b2ee2531c4 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3.sh @@ -28,14 +28,14 @@ if [[ -z "$IT_CASE_S3_ACCESS_KEY" ]]; then echo "Did not find AWS environment variables, NOT running the e2e test." exit 0 else - echo "Found AWS access key $IT_CASE_S3_ACCESS_KEY, running the e2e test." + echo "Found AWS access key, running the e2e test." fi if [[ -z "$IT_CASE_S3_SECRET_KEY" ]]; then echo "Did not find AWS environment variables, NOT running the e2e test." exit 0 else - echo "Found AWS secret key $IT_CASE_S3_SECRET_KEY, running the e2e test." + echo "Found AWS secret key, running the e2e test." fi AWS_REGION="${AWS_REGION:-us-east-1}" From 9517201512167c222365d0fd0cd2812fc97863a2 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 16 Nov 2018 18:11:36 +0100 Subject: [PATCH 104/359] [FLINK-10869] [tests] Fix test_streaming_file_sink to use proper and unique output directory --- .../test-scripts/test_streaming_file_sink.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh index 7a469aa7951ae..e810e68bde740 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_file_sink.sh @@ -24,9 +24,9 @@ source "$(dirname "$0")"/common_s3.sh set_conf_ssl "mutual" -OUT=out +OUT=temp/test_streaming_file_sink-$(uuidgen) OUTPUT_PATH="$TEST_DATA_DIR/$OUT" -S3_OUTPUT_PATH="s3://$IT_CASE_S3_BUCKET/temp/$OUT" +S3_OUTPUT_PATH="s3://$IT_CASE_S3_BUCKET/$OUT" mkdir -p $OUTPUT_PATH From 6abd87b274dbb9f1b5956965789a10ec5b6cfba8 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 13 Nov 2018 18:21:04 +0100 Subject: [PATCH 105/359] [hotfix] [tests] Make S3 Recoverable Writer tests ITCases --- ...nTest.java => HadoopS3RecoverableWriterExceptionITCase.java} | 2 +- ...ableWriterTest.java => HadoopS3RecoverableWriterITCase.java} | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/{HadoopS3RecoverableWriterExceptionTest.java => HadoopS3RecoverableWriterExceptionITCase.java} (98%) rename flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/{HadoopS3RecoverableWriterTest.java => HadoopS3RecoverableWriterITCase.java} (99%) diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionITCase.java similarity index 98% rename from flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java rename to flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionITCase.java index 3fb67c1414f5a..59b8b752a7084 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionTest.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterExceptionITCase.java @@ -51,7 +51,7 @@ * Tests for exception throwing in the * {@link org.apache.flink.fs.s3.common.writer.S3RecoverableWriter S3RecoverableWriter}. */ -public class HadoopS3RecoverableWriterExceptionTest extends TestLogger { +public class HadoopS3RecoverableWriterExceptionITCase extends TestLogger { // ----------------------- S3 general configuration ----------------------- diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java similarity index 99% rename from flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java rename to flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java index a9027a0b62c8e..17fb02b188cc0 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterTest.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java @@ -59,7 +59,7 @@ /** * Tests for the {@link org.apache.flink.fs.s3.common.writer.S3RecoverableWriter S3RecoverableWriter}. */ -public class HadoopS3RecoverableWriterTest extends TestLogger { +public class HadoopS3RecoverableWriterITCase extends TestLogger { // ----------------------- S3 general configuration ----------------------- From 376cc227e974f87a545d1044418563523387cd33 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 17:05:16 +0100 Subject: [PATCH 106/359] [hotfix] [s3] Minor fixes/code simplifications in S3 recoverable writer --- .../common/utils/BackPressuringExecutor.java | 5 +--- .../fs/s3/common/writer/S3ConfigOptions.java | 30 ------------------- 2 files changed, 1 insertion(+), 34 deletions(-) delete mode 100644 flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3ConfigOptions.java diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/BackPressuringExecutor.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/BackPressuringExecutor.java index d0dd7c8f73440..e78e83de4252e 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/BackPressuringExecutor.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/BackPressuringExecutor.java @@ -23,7 +23,6 @@ import org.apache.flink.util.FlinkRuntimeException; import java.util.concurrent.Executor; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; @@ -66,11 +65,9 @@ public void execute(Runnable command) { final SemaphoreReleasingRunnable runnable = new SemaphoreReleasingRunnable(command, permits); try { delegate.execute(runnable); - } catch (RejectedExecutionException e) { + } catch (Throwable e) { runnable.release(); ExceptionUtils.rethrow(e, e.getMessage()); - } catch (Throwable t) { - ExceptionUtils.rethrow(t, t.getMessage()); } } diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3ConfigOptions.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3ConfigOptions.java deleted file mode 100644 index 421880554e5e8..0000000000000 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3ConfigOptions.java +++ /dev/null @@ -1,30 +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.fs.s3.common.writer; - -/** - * Configuration keys for the S3 file system based using Hadoop's s3a. - */ -public final class S3ConfigOptions { - - // ------------------------------------------------------------------------ - - /** Not meant to be instantiated. */ - private S3ConfigOptions() {} -} From 44e60619226e5d5aa3004eeca2a5e9b0f37904e6 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 20:49:04 +0100 Subject: [PATCH 107/359] [hotfix] [tests] Make S3 config key forwarding a proper unit test This avoids unnecessary and expensive connections to S3 just to validate whether config keys of various formats are forwarded. --- .../fs/s3hadoop/HadoopS3FileSystemITCase.java | 35 ------------- .../fs/s3hadoop/HadoopS3FileSystemTest.java | 51 +++++++++++++++++++ 2 files changed, 51 insertions(+), 35 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java index 2195bd0e7632c..1fd0afd5c8636 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java @@ -125,41 +125,6 @@ private String getBasePath() { return S3TestCredentials.getTestBucketUriWithScheme(scheme) + TEST_DATA_DIR + '/' + scheme; } - @Test - public void testConfigKeysForwarding() throws Exception { - final Path path = new Path(getBasePath()); - - // standard Hadoop-style credential keys - { - Configuration conf = new Configuration(); - conf.setString("fs.s3a.access.key", S3TestCredentials.getS3AccessKey()); - conf.setString("fs.s3a.secret.key", S3TestCredentials.getS3SecretKey()); - - FileSystem.initialize(conf); - path.getFileSystem(); - } - - // shortened Hadoop-style credential keys - { - Configuration conf = new Configuration(); - conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); - conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); - - FileSystem.initialize(conf); - path.getFileSystem(); - } - - // shortened Presto-style credential keys - { - Configuration conf = new Configuration(); - conf.setString("s3.access-key", S3TestCredentials.getS3AccessKey()); - conf.setString("s3.secret-key", S3TestCredentials.getS3SecretKey()); - - FileSystem.initialize(conf); - path.getFileSystem(); - } - } - @Test public void testSimpleFileWriteAndRead() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemTest.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemTest.java index 6faf5b2575f7b..4471b3868dfdd 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemTest.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemTest.java @@ -29,6 +29,7 @@ * Unit tests for the S3 file system support via Hadoop's {@link org.apache.hadoop.fs.s3a.S3AFileSystem}. */ public class HadoopS3FileSystemTest { + @Test public void testShadingOfAwsCredProviderConfig() { final Configuration conf = new Configuration(); @@ -41,4 +42,54 @@ public void testShadingOfAwsCredProviderConfig() { assertEquals("org.apache.flink.fs.s3hadoop.shaded.com.amazonaws.auth.ContainerCredentialsProvider", hadoopConfig.get("fs.s3a.aws.credentials.provider")); } + + // ------------------------------------------------------------------------ + // These tests check that the S3FileSystemFactory properly forwards + // various patterns of keys for credentials. + // ------------------------------------------------------------------------ + + /** + * Test forwarding of standard Hadoop-style credential keys. + */ + @Test + public void testConfigKeysForwardingHadoopStyle() { + Configuration conf = new Configuration(); + conf.setString("fs.s3a.access.key", "test_access_key"); + conf.setString("fs.s3a.secret.key", "test_secret_key"); + + checkHadoopAccessKeys(conf, "test_access_key", "test_secret_key"); + } + + /** + * Test forwarding of shortened Hadoop-style credential keys. + */ + @Test + public void testConfigKeysForwardingShortHadoopStyle() { + Configuration conf = new Configuration(); + conf.setString("s3.access.key", "my_key_a"); + conf.setString("s3.secret.key", "my_key_b"); + + checkHadoopAccessKeys(conf, "my_key_a", "my_key_b"); + } + + /** + * Test forwarding of shortened Presto-style credential keys. + */ + @Test + public void testConfigKeysForwardingPrestoStyle() { + Configuration conf = new Configuration(); + conf.setString("s3.access-key", "clé d'accès"); + conf.setString("s3.secret-key", "clef secrète"); + checkHadoopAccessKeys(conf, "clé d'accès", "clef secrète"); + } + + private static void checkHadoopAccessKeys(Configuration flinkConf, String accessKey, String secretKey) { + HadoopConfigLoader configLoader = S3FileSystemFactory.createHadoopConfigLoader(); + configLoader.setFlinkConfig(flinkConf); + + org.apache.hadoop.conf.Configuration hadoopConf = configLoader.getOrLoadHadoopConfig(); + + assertEquals(accessKey, hadoopConf.get("fs.s3a.access.key", null)); + assertEquals(secretKey, hadoopConf.get("fs.s3a.secret.key", null)); + } } From f3dc54c7775aa67796fd6edd82f41a5ec7b85139 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 21:17:13 +0100 Subject: [PATCH 108/359] [hotfix] [tests] Check for s3a and s3 schemes in a unit test This avoid duplicating the very expensive and time consuming execution of the S3 file system integration tests. The scheme is an artifact purely of the factory with no need to test it in an integration test of the actual file system. --- .../fs/s3hadoop/HadoopS3FileSystemITCase.java | 15 +---- .../HadoopS3FileSystemsSchemesTest.java | 55 +++++++++++++++++++ 2 files changed, 56 insertions(+), 14 deletions(-) create mode 100644 flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemsSchemesTest.java diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java index 1fd0afd5c8636..3f09330bc5898 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java @@ -31,16 +31,12 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.List; import java.util.UUID; import static org.apache.flink.core.fs.FileSystemTestUtils.checkPathEventualExistence; @@ -56,17 +52,8 @@ * consistency guarantees * and what the {@link org.apache.hadoop.fs.s3a.S3AFileSystem} offers. */ -@RunWith(Parameterized.class) public class HadoopS3FileSystemITCase extends TestLogger { - @Parameterized.Parameter - public String scheme; - - @Parameterized.Parameters(name = "Scheme = {0}") - public static List parameters() { - return Arrays.asList("s3", "s3a"); - } - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); /** @@ -122,7 +109,7 @@ public static void cleanUp() throws IOException, InterruptedException { } private String getBasePath() { - return S3TestCredentials.getTestBucketUriWithScheme(scheme) + TEST_DATA_DIR + '/' + scheme; + return S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR; } @Test diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemsSchemesTest.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemsSchemesTest.java new file mode 100644 index 0000000000000..a57b4da35e852 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemsSchemesTest.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.fs.s3hadoop; + +import org.apache.flink.core.fs.FileSystemFactory; + +import org.junit.Test; + +import java.util.ServiceLoader; + +import static org.junit.Assert.fail; + +/** + * This test validates that the S3 file system registers both under s3:// and s3a://. + */ +public class HadoopS3FileSystemsSchemesTest { + + @Test + public void testS3Factory() { + testFactory("s3"); + } + + @Test + public void testS3AFactory() { + testFactory("s3a"); + } + + private static void testFactory(String scheme) { + ServiceLoader serviceLoader = ServiceLoader.load(FileSystemFactory.class); + for (FileSystemFactory fs : serviceLoader) { + if (scheme.equals(fs.getScheme())) { + // found the matching scheme + return; + } + } + + fail("No factory available for scheme " + scheme); + } +} From fad09e49e918863c6b930fd3f99527a707987250 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 15 Nov 2018 21:30:31 +0100 Subject: [PATCH 109/359] [hotfix] [tests] Simplify and speed up S3 filesystem tests Rather then reinitializing the S3 file system in each test, use one file system across tests. This reduces code and speeds up the test. --- .../fs/s3hadoop/HadoopS3FileSystemITCase.java | 49 +++++-------------- 1 file changed, 12 insertions(+), 37 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java index 3f09330bc5898..70b5bce10a5b1 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemITCase.java @@ -54,12 +54,12 @@ */ public class HadoopS3FileSystemITCase extends TestLogger { - private static final String TEST_DATA_DIR = "tests-" + UUID.randomUUID(); - /** * Will be updated by {@link #checkCredentialsAndSetup()} if the test is not skipped. */ - private static boolean skipTest = true; + private static FileSystem fileSystem; + + private static Path basePath; @BeforeClass public static void checkCredentialsAndSetup() throws IOException { @@ -72,35 +72,25 @@ public static void checkCredentialsAndSetup() throws IOException { conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); FileSystem.initialize(conf); - // check for uniqueness of the test directory - final Path directory = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); - final FileSystem fs = directory.getFileSystem(); + basePath = new Path(S3TestCredentials.getTestBucketUri() + "tests-" + UUID.randomUUID()); + fileSystem = basePath.getFileSystem(); + // check for uniqueness of the test directory // directory must not yet exist - assertFalse(fs.exists(directory)); - - skipTest = false; + assertFalse(fileSystem.exists(basePath)); } @AfterClass public static void cleanUp() throws IOException, InterruptedException { try { - if (!skipTest) { + if (fileSystem != null) { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs - // initialize configuration with valid credentials - final Configuration conf = new Configuration(); - conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); - conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); - FileSystem.initialize(conf); - - final Path directory = new Path(S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR); - final FileSystem fs = directory.getFileSystem(); // clean up - fs.delete(directory, true); + fileSystem.delete(basePath, true); // now directory must be gone - checkPathEventualExistence(fs, directory, false, deadline); + checkPathEventualExistence(fileSystem, basePath, false, deadline); } } finally { @@ -108,22 +98,12 @@ public static void cleanUp() throws IOException, InterruptedException { } } - private String getBasePath() { - return S3TestCredentials.getTestBucketUri() + TEST_DATA_DIR; - } - @Test public void testSimpleFileWriteAndRead() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs - final Configuration conf = new Configuration(); - conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); - conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); - final String testLine = "Hello Upload!"; - FileSystem.initialize(conf); - - final Path path = new Path(getBasePath() + "/test.txt"); + final Path path = new Path(basePath, "test.txt"); final FileSystem fs = path.getFileSystem(); try { @@ -153,13 +133,8 @@ public void testSimpleFileWriteAndRead() throws Exception { @Test public void testDirectoryListing() throws Exception { final long deadline = System.nanoTime() + 30_000_000_000L; // 30 secs - final Configuration conf = new Configuration(); - conf.setString("s3.access.key", S3TestCredentials.getS3AccessKey()); - conf.setString("s3.secret.key", S3TestCredentials.getS3SecretKey()); - - FileSystem.initialize(conf); - final Path directory = new Path(getBasePath() + "/testdir/"); + final Path directory = new Path(basePath, "testdir/"); final FileSystem fs = directory.getFileSystem(); // directory must not yet exist From db9b5385d8cca9b839fce9247fd70d9ccafd99c8 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 16 Nov 2018 18:04:24 +0100 Subject: [PATCH 110/359] [FLINK-10736] [tests] Use staticly hosted test data for S3 wordcount tests This avoids issues with eventual consistency/visibility on S3. --- flink-end-to-end-tests/test-scripts/common_s3.sh | 2 ++ .../test-scripts/test_shaded_hadoop_s3a.sh | 9 +-------- .../test-scripts/test_shaded_presto_s3.sh | 9 +-------- 3 files changed, 4 insertions(+), 16 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3.sh b/flink-end-to-end-tests/test-scripts/common_s3.sh index 1f3b2ee2531c4..edb289506ec87 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3.sh @@ -42,6 +42,8 @@ AWS_REGION="${AWS_REGION:-us-east-1}" AWS_ACCESS_KEY=$IT_CASE_S3_ACCESS_KEY AWS_SECRET_KEY=$IT_CASE_S3_SECRET_KEY +S3_TEST_DATA_WORDS_URI="s3://$IT_CASE_S3_BUCKET/static/words" + s3util="java -jar ${END_TO_END_DIR}/flink-e2e-test-utils/target/S3UtilProgram.jar" ################################### diff --git a/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh index e6ae9929aef23..489d0df6ade4e 100755 --- a/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh +++ b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh @@ -22,15 +22,8 @@ source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/common_s3.sh -s3_put $TEST_INFRA_DIR/test-data/words $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-s3a -# make sure we delete the file at the end -function shaded_s3a_cleanup { - s3_delete $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-s3a -} -trap shaded_s3a_cleanup EXIT - start_cluster -$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out +$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input $S3_TEST_DATA_WORDS_URI --output $TEST_DATA_DIR/out/wc_out check_result_hash "WordCountWithShadedS3A" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" diff --git a/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh index 0421c840a3dc6..a963e0961319f 100755 --- a/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh +++ b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh @@ -22,15 +22,8 @@ source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/common_s3.sh -s3_put $TEST_INFRA_DIR/test-data/words $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-presto-s3 -# make sure we delete the file at the end -function shaded_presto_s3_cleanup { - s3_delete $IT_CASE_S3_BUCKET temp/flink-end-to-end-test-shaded-presto-s3 -} -trap shaded_presto_s3_cleanup EXIT - start_cluster -$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input s3:/$resource --output $TEST_DATA_DIR/out/wc_out +$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input $S3_TEST_DATA_WORDS_URI --output $TEST_DATA_DIR/out/wc_out check_result_hash "WordCountWithShadedPrestoS3" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5" From 1acf5ee2a4beb11a0460d71e2052060d0ddba4f0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Nov 2018 16:38:55 +0100 Subject: [PATCH 111/359] [FLINK-10906][docker] Don't print configuration in docker-entrypoint.sh In order to not leak secrets we should not print the configuration in docker-entrypoint.sh. --- flink-container/docker/docker-entrypoint.sh | 1 - flink-contrib/docker-flink/docker-entrypoint.sh | 2 -- 2 files changed, 3 deletions(-) diff --git a/flink-container/docker/docker-entrypoint.sh b/flink-container/docker/docker-entrypoint.sh index 85ba8662bee2b..0bf7c04fa93c0 100755 --- a/flink-container/docker/docker-entrypoint.sh +++ b/flink-container/docker/docker-entrypoint.sh @@ -32,7 +32,6 @@ if [ "${CMD}" == "--help" -o "${CMD}" == "-h" ]; then exit 0 elif [ "${CMD}" == "${JOB_CLUSTER}" -o "${CMD}" == "${TASK_MANAGER}" ]; then echo "Starting the ${CMD}" - echo "config file: " && grep '^[^\n#]' $FLINK_HOME/conf/flink-conf.yaml if [ "${CMD}" == "${TASK_MANAGER}" ]; then exec $FLINK_HOME/bin/taskmanager.sh start-foreground "$@" diff --git a/flink-contrib/docker-flink/docker-entrypoint.sh b/flink-contrib/docker-flink/docker-entrypoint.sh index 1edf99ca6ab9f..a4e279fad3bdc 100755 --- a/flink-contrib/docker-flink/docker-entrypoint.sh +++ b/flink-contrib/docker-flink/docker-entrypoint.sh @@ -29,7 +29,6 @@ elif [ "$1" == "jobmanager" ]; then echo "Starting Job Manager" sed -i -e "s/jobmanager.rpc.address: localhost/jobmanager.rpc.address: ${JOB_MANAGER_RPC_ADDRESS}/g" $FLINK_HOME/conf/flink-conf.yaml - echo "config file: " && grep '^[^\n#]' $FLINK_HOME/conf/flink-conf.yaml exec $FLINK_HOME/bin/jobmanager.sh start-foreground elif [ "$1" == "taskmanager" ]; then @@ -37,7 +36,6 @@ elif [ "$1" == "taskmanager" ]; then sed -i -e "s/taskmanager.numberOfTaskSlots: 1/taskmanager.numberOfTaskSlots: $(grep -c ^processor /proc/cpuinfo)/g" $FLINK_HOME/conf/flink-conf.yaml echo "Starting Task Manager" - echo "config file: " && grep '^[^\n#]' $FLINK_HOME/conf/flink-conf.yaml exec $FLINK_HOME/bin/taskmanager.sh start-foreground fi From 1b8b7ae5930ce9d5625e7590cb40b3f1799aa84b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 17 Nov 2018 11:42:42 +0100 Subject: [PATCH 112/359] [FLINK-10913] Harden ExecutionGraphRestartTest#testRestartAutomatically Wait until all Executions reach the state DEPLOYING instead of having a resource assigned. --- .../executiongraph/ExecutionGraphRestartTest.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 91510d1af547c..ffe1fd004812a 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 @@ -973,7 +973,7 @@ private static void restartAfterFailure(ExecutionGraph eg, FiniteDuration timeou // Wait for deploying after async restart Deadline deadline = timeout.fromNow(); - waitForAllResourcesToBeAssignedAfterAsyncRestart(eg, deadline); + waitUntilAllExecutionsReachDeploying(eg, deadline); if (haltAfterRestart) { if (deadline.hasTimeLeft()) { @@ -984,6 +984,13 @@ private static void restartAfterFailure(ExecutionGraph eg, FiniteDuration timeou } } + private static void waitUntilAllExecutionsReachDeploying(ExecutionGraph eg, Deadline deadline) throws TimeoutException { + ExecutionGraphTestUtils.waitForAllExecutionsPredicate( + eg, + ExecutionGraphTestUtils.isInExecutionState(ExecutionState.DEPLOYING), + deadline.timeLeft().toMillis()); + } + private static void waitForAllResourcesToBeAssignedAfterAsyncRestart(ExecutionGraph eg, Deadline deadline) throws TimeoutException { ExecutionGraphTestUtils.waitForAllExecutionsPredicate( eg, From ac3093847332dd6790d3f78a38f149a08e7ccf1b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Nov 2018 17:47:45 +0100 Subject: [PATCH 113/359] [FLINK-10880] Add Documentation.ExcludeFromDocumentation to exclude ConfigOptions from documentation The annotation Documentation.ExcludeFromDocumentation can be used to annotate ConfigOptions with in order to not include them in the documentation. --- .../flink/annotation/docs/Documentation.java | 13 ++++++ .../ConfigOptionsDocGenerator.java | 11 ++++- .../ConfigOptionsDocGeneratorTest.java | 41 +++++++++++++++++++ 3 files changed, 64 insertions(+), 1 deletion(-) diff --git a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java index c193c4e055499..aed0d2975f490 100644 --- a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java +++ b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java @@ -59,6 +59,19 @@ public final class Documentation { int position() default Integer.MAX_VALUE; } + /** + * Annotation used on config option fields to exclude the config option from documentation. + */ + @Target(ElementType.FIELD) + @Retention(RetentionPolicy.RUNTIME) + @Internal + public @interface ExcludeFromDocumentation { + /** + * The optional reason why the config option is excluded from documentation. + */ + String value() default ""; + } + private Documentation(){ } } diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index d0f665a5051e4..36a2203b7e4f7 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -202,7 +202,7 @@ static List extractConfigOptions(Class clazz) { List configOptions = new ArrayList<>(8); Field[] fields = clazz.getFields(); for (Field field : fields) { - if (field.getType().equals(ConfigOption.class) && field.getAnnotation(Deprecated.class) == null) { + if (isConfigOption(field) && shouldBeDocumented(field)) { configOptions.add(new OptionWithMetaInfo((ConfigOption) field.get(null), field)); } } @@ -213,6 +213,15 @@ static List extractConfigOptions(Class clazz) { } } + private static boolean isConfigOption(Field field) { + return field.getType().equals(ConfigOption.class); + } + + private static boolean shouldBeDocumented(Field field) { + return field.getAnnotation(Deprecated.class) == null && + field.getAnnotation(Documentation.ExcludeFromDocumentation.class) == null; + } + /** * Transforms this configuration group into HTML formatted table. * Options are sorted alphabetically by key. diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java index a0eade69d455f..c0fb7c99eda73 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java @@ -266,4 +266,45 @@ public void testCommonOptions() throws IOException, ClassNotFoundException { assertEquals(expected, output); } + + static class TestConfigGroupWithExclusion { + public static ConfigOption firstOption = ConfigOptions + .key("first.option.a") + .defaultValue(2) + .withDescription("This is example description for the first option."); + + @Documentation.ExcludeFromDocumentation + public static ConfigOption excludedOption = ConfigOptions + .key("excluded.option.a") + .noDefaultValue() + .withDescription("This should not be documented."); + } + + /** + * Tests that {@link ConfigOption} annotated with {@link Documentation.ExcludeFromDocumentation} + * are not documented. + */ + @Test + public void testConfigOptionExclusion() { + final String expectedTable = + "\n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + " \n" + + "
KeyDefaultDescription
first.option.a
2This is example description for the first option.
\n"; + final String htmlTable = ConfigOptionsDocGenerator.generateTablesForClass(TestConfigGroupWithExclusion.class).get(0).f1; + + assertEquals(expectedTable, htmlTable); + } } From 63d388321681f3a205b62539b71d400700fe1ab2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Nov 2018 17:53:22 +0100 Subject: [PATCH 114/359] [FLINK-10880] Exclude JobManagerOptions#EXECUTION_FAILOVER_STRATEGY from documentation This commit excludes the JobManagerOptions#EXECUTION_FAILOVER_STRATEGY from Flink's configuration documentation. --- docs/_includes/generated/job_manager_configuration.html | 5 ----- .../org/apache/flink/configuration/JobManagerOptions.java | 1 + 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/_includes/generated/job_manager_configuration.html b/docs/_includes/generated/job_manager_configuration.html index 0458af24c06eb..99eec1dcec7af 100644 --- a/docs/_includes/generated/job_manager_configuration.html +++ b/docs/_includes/generated/job_manager_configuration.html @@ -17,11 +17,6 @@ 16 The maximum number of prior execution attempts kept in history. - -

jobmanager.execution.failover-strategy
- "full" - This option specifies how the job computation recovers from task failures. Accepted values are:
  • 'full': Restarts all tasks.
  • 'individual': Restarts only the failed task. Should only be used if all tasks are independent components.
  • 'region': Restarts all tasks that could be affected by the task failure.
-
jobmanager.heap.size
"1024m" diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index 1666f213d182e..cb2dbe6163189 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -106,6 +106,7 @@ public class JobManagerOptions { /** * This option specifies the failover strategy, i.e. how the job computation recovers from task failures. */ + @Documentation.ExcludeFromDocumentation("The failover strategy feature is highly experimental.") public static final ConfigOption EXECUTION_FAILOVER_STRATEGY = key("jobmanager.execution.failover-strategy") .defaultValue("full") From ef683c3ea56b5e82971035790955d895d93ccd5a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Nov 2018 18:32:21 +0100 Subject: [PATCH 115/359] [FLINK-10880] Add release notes warning to not use Flink's failover strategies --- docs/release-notes/flink-1.5.md | 8 ++++++++ docs/release-notes/flink-1.6.md | 11 ++++++++++- docs/release-notes/flink-1.7.md | 9 +++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/docs/release-notes/flink-1.5.md b/docs/release-notes/flink-1.5.md index 4cee5774cdffc..ed5f2c2ad2654 100644 --- a/docs/release-notes/flink-1.5.md +++ b/docs/release-notes/flink-1.5.md @@ -80,5 +80,13 @@ The Kinesis dependencies of Flink’s Kinesis connector have been updated to the 0.12.9 ``` + +### Limitations of failover strategies +Flink's non-default failover strategies are still a very experimental feature which come with a set of limitations. +You should only use this feature if you are executing a stateless streaming job. +In any other cases, it is highly recommended to remove the config option `jobmanager.execution.failover-strategy` from your `flink-conf.yaml` or set it to `"full"`. + +In order to avoid future problems, this feature has been removed from the documentation until it will be fixed. +See [FLINK-10880](https://issues.apache.org/jira/browse/FLINK-10880) for more details. {% top %} diff --git a/docs/release-notes/flink-1.6.md b/docs/release-notes/flink-1.6.md index 34cd613551106..e2f9cc110d21c 100644 --- a/docs/release-notes/flink-1.6.md +++ b/docs/release-notes/flink-1.6.md @@ -29,6 +29,15 @@ The default value of the slot idle timeout `slot.idle.timeout` is set to the def ### Changed ElasticSearch 5.x Sink API Previous APIs in the Flink ElasticSearch 5.x Sink's `RequestIndexer` interface have been deprecated in favor of new signatures. -When adding requests to the `RequestIndexer`, the requests now must be of type `IndexRequest`, `DeleteRequest`, or `UpdateRequest`, instead of the base `ActionRequest`. +When adding requests to the `RequestIndexer`, the requests now must be of type `IndexRequest`, `DeleteRequest`, or `UpdateRequest`, instead of the base `ActionRequest`. + + +### Limitations of failover strategies +Flink's non-default failover strategies are still a very experimental feature which come with a set of limitations. +You should only use this feature if you are executing a stateless streaming job. +In any other cases, it is highly recommended to remove the config option `jobmanager.execution.failover-strategy` from your `flink-conf.yaml` or set it to `"full"`. + +In order to avoid future problems, this feature has been removed from the documentation until it will be fixed. +See [FLINK-10880](https://issues.apache.org/jira/browse/FLINK-10880) for more details. {% top %} diff --git a/docs/release-notes/flink-1.7.md b/docs/release-notes/flink-1.7.md index f9e7425601a42..b26ef8695b772 100644 --- a/docs/release-notes/flink-1.7.md +++ b/docs/release-notes/flink-1.7.md @@ -30,4 +30,13 @@ Therefore, the module `flink-scala-shell` is not being released for Scala 2.12. See [FLINK-10911](https://issues.apache.org/jira/browse/FLINK-10911) for more details. + +### Limitations of failover strategies +Flink's non-default failover strategies are still a very experimental feature which come with a set of limitations. +You should only use this feature if you are executing a stateless streaming job. +In any other cases, it is highly recommended to remove the config option `jobmanager.execution.failover-strategy` from your `flink-conf.yaml` or set it to `"full"`. + +In order to avoid future problems, this feature has been removed from the documentation until it will be fixed. +See [FLINK-10880](https://issues.apache.org/jira/browse/FLINK-10880) for more details. + {% top %} From d61f3bc899a53fd77875686fe62ef152c68e10dc Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 9 Nov 2018 12:40:21 +0100 Subject: [PATCH 116/359] [FLINK-10625] [docs] Documentation for MATCH_RECOGNIZE clause --- docs/dev/table/sql.md | 83 ++ docs/dev/table/streaming/match_recognize.md | 732 ++++++++++++++++++ .../table/streaming/query_configuration.md | 2 +- 3 files changed, 816 insertions(+), 1 deletion(-) create mode 100644 docs/dev/table/streaming/match_recognize.md diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index b1bd572d4d2f6..fbd755793e970 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -163,6 +163,7 @@ joinCondition: tableReference: tablePrimary + [ matchRecognize ] [ [ AS ] alias [ '(' columnAlias [, columnAlias ]* ')' ] ] tablePrimary: @@ -196,6 +197,45 @@ windowSpec: ] ')' +matchRecognize: + MATCH_RECOGNIZE '(' + [ PARTITION BY expression [, expression ]* ] + [ ORDER BY orderItem [, orderItem ]* ] + [ MEASURES measureColumn [, measureColumn ]* ] + [ ONE ROW PER MATCH ] + [ AFTER MATCH + ( SKIP TO NEXT ROW + | SKIP PAST LAST ROW + | SKIP TO FIRST variable + | SKIP TO LAST variable + | SKIP TO variable ) + ] + PATTERN '(' pattern ')' + DEFINE variable AS condition [, variable AS condition ]* + ')' + +measureColumn: + expression AS alias + +pattern: + patternTerm [ '|' patternTerm ]* + +patternTerm: + patternFactor [ patternFactor ]* + +patternFactor: + variable [ patternQuantifier ] + +patternQuantifier: + '*' + | '*?' + | '+' + | '+?' + | '?' + | '??' + | '{' { [ minRepeat ], [ maxRepeat ] } '}' ['?'] + | '{' repeat '}' + {% endhighlight %} Flink SQL uses a lexical policy for identifier (table, attribute, function names) similar to Java: @@ -756,6 +796,49 @@ Group windows are defined in the `GROUP BY` clause of a SQL query. Just like que +### Pattern Recognition + +
+ + + + + + + + + + + + + + +
OperationDescription
+ MATCH_RECOGNIZE
+ Streaming +
+

Searches for a given pattern in a streaming table according to the MATCH_RECOGNIZE standard). For a more detailed description see Detecting patterns

+ +{% highlight sql %} +SELECT T.aid, T.bid, T.cid +FROM MyTable +MATCH_RECOGNIZE ( + ORDER BY proctime + MEASURES + A.id AS aid, + B.id AS bid, + C.id AS cid + PATTERN (A B C) + DEFINE + A AS name = 'a', + B AS name = 'b', + C AS name = 'c' +) AS T +{% endhighlight %} +
+
+ +{% top %} #### Time Attributes diff --git a/docs/dev/table/streaming/match_recognize.md b/docs/dev/table/streaming/match_recognize.md new file mode 100644 index 0000000000000..bb8e5578d83a9 --- /dev/null +++ b/docs/dev/table/streaming/match_recognize.md @@ -0,0 +1,732 @@ +--- +title: 'Detecting patterns' +nav-parent_id: streaming_tableapi +nav-title: 'Detecting patterns' +nav-pos: 5 +is_beta: true +--- + + +It is a common use-case to search for a set of event patterns, especially in case of data streams. Apache Flink +comes with [CEP library]({{ site.baseurl }}/dev/libs/cep.html) which allows pattern detection in event streams. On the other hand Flink's +SQL API provides a relational way to express queries that comes with multiple functions and +optimizations that can be used out of the box. In December 2016, ISO released a new version of the +international SQL standard ([ISO/IEC 9075:2016](https://standards.iso.org/ittf/PubliclyAvailableStandards/c065143_ISO_IEC_TR_19075-5_2016.zip)) +including the Row Pattern Recognition for complex event processing, which allowes to consolidate those two APIs using `MATCH_RECOGNIZE` clause. + +`MATCH_RECOGNIZE` enables you to do the following tasks: +* Logically partition and order the data that is used with `PARTITION BY` and `ORDER BY` clauses. +* Define patterns of rows to seek using the `PATTERN` clause. These patterns use a powerful and expressive regular expression syntax. +* Specify logical conditions required to map a row to a row pattern variable in the `DEFINE` clause. +* Define measures, which are expressions usable in other parts of the SQL query, in the `MEASURES` clause. + +Every `MATCH_RECOGNIZE` query consists of the following clauses: + +* [PARTITION BY](#partitioning) - defines logical partitioning of the stream, similar to `GROUP BY` operation. +* [ORDER BY](#order-of-events) - specifies how the incoming events should be ordered, this is essential as patterns depend on order. +* [MEASURES](#define--measures) - defines output of the clause, similar to `SELECT` clause +* [ONE ROW PER MATCH](#output-mode) - output mode which defines how many rows per match should be produced +* [AFTER MATCH SKIP](#after-match-skip) - allows to specify where the next match should start, this is also a way to control how many distinct matches a single event can belong to +* [PATTERN](#defining-pattern) - allows constructing patterns that will be searched for using a regular expression like syntax +* [DEFINE](#define--measures) - this section defines conditions on events that should be met in order to be qualified to the corresponding pattern variable + + +This clause can only be applied to an [append](dynamic_tables.html#update-and-append-queries) table and it always produces +an append table as well. + +* This will be replaced by the TOC +{:toc} + +Example query +------------- + +Having a table `TICKER`, describing prices of stocks in a particular moment of time, each row represents an updated characteristic of a ticker. +The table has a following schema: + +{% highlight text %} +Ticker + |-- symbol: Long # symbol of the stock + |-- price: Long # price of the stock + |-- tax: Long # tax liability of the stock + |-- rowTime: TimeIndicatorTypeInfo(rowtime) # point in time when change to those values happened +{% endhighlight %} + +The incoming data for a single ticker could look following: + +{% highlight text %} +SYMBOL ROWTIME PRICE TAX +====== ==================== ======= ======= +'ACME' '01-Apr-11 10:00:00' 12 1 +'ACME' '01-Apr-11 10:00:01' 17 2 +'ACME' '01-Apr-11 10:00:02' 19 1 +'ACME' '01-Apr-11 10:00:03' 21 3 +'ACME' '01-Apr-11 10:00:04' 25 2 +'ACME' '01-Apr-11 10:00:05' 18 1 +'ACME' '01-Apr-11 10:00:06' 15 1 +'ACME' '01-Apr-11 10:00:07' 14 2 +'ACME' '01-Apr-11 10:00:08' 24 2 +'ACME' '01-Apr-11 10:00:09' 25 2 +'ACME' '01-Apr-11 10:00:10' 19 1 +{% endhighlight %} + +For example to find periods of a constantly decreasing price of a single ticker one could write a query like this: + +{% highlight sql %} +SELECT * +FROM Ticker +MATCH_RECOGNIZE ( + PARTITION BY symbol + ORDER BY rowtime + MEASURES + STRT_ROW.rowtime AS start_tstamp, + LAST(PRICE_DOWN.rowtime) AS bottom_tstamp, + LAST(PRICE_UP.rowtime) AS end_tstamp + ONE ROW PER MATCH + AFTER MATCH SKIP TO LAST UP + PATTERN (STRT_ROW PRICE_DOWN+ PRICE_UP) + DEFINE + PRICE_DOWN AS PRICE_DOWN.price < LAST(PRICE_DOWN.price, 1) OR + (LAST(PRICE_DOWN.price, 1) IS NULL AND PRICE_DOWN.price < STRT_ROW.price)) + PRICE_UP AS PRICE_UP.price > LAST(PRICE_DOWN.price, 1) + ) MR; +{% endhighlight %} + +which will produce a summary row for each found period in which the price was constantly decreasing. + +{% highlight text %} +SYMBOL START_TST BOTTOM_TS END_TSTAM +========= ================== ================== ================== +ACME 01-APR-11 10:00:04 01-APR-11 10:00:07 01-APR-11 10:00:08 +{% endhighlight %} + +The resulting row says that the query found a period of a decreasing price for a ticker that started at `01-APR-11 10:00:04`, +achieved the lowest value at `01-APR-11 10:00:07` and the price increased at `01-APR-11 10:00:08` + +Installation guide +------------------ + +Pattern recognition feature uses the Apache Flink's CEP library internally. In order to be able to use this clause one has to add +the library as a dependency. Either by adding it to your uber-jar or by adding a dependency on: + +{% highlight xml %} + + org.apache.flink + flink-cep{{ site.scala_version_suffix }} + {{ site.version }} + +{% endhighlight %} + +or by adding it to the cluster classpath (see [here]({{ site.baseurl}}/dev/linking.html)). If you want to use +the `MATCH_RECOGNIZE` clause from [sql-client]({{ site.baseurl}}/dev/table/sqlClient.html) you don't have to do anything as all the dependencies are included by default. + +Partitioning +------------ +It is possible to look for patterns in a partitioned data, e.g. trends for a single ticker. This can be expressed using the `PARTITION BY` clause. It is similar to using the `GROUP BY` for aggregations. + +Attention It is highly advised to partition the incoming data because otherwise the `MATCH_RECOGNIZE` will be translated +into a non-parallel operator to ensure global ordering. + +Order of events +--------------- + +Apache Flink allows searching for patterns based on time, either [processing-time or event-time](time_attributes.html). This assumption +is very important, because it allows sorting events before they are passed to pattern state machine. This ensures +that the produced output will be correct in regards not to the order the rows arrived, but in which the event corresponding to the row really happened. + +As a consequence one has to provide a time attribute with ascending ordering as the first argument to `ORDER BY` clause. +That means for the example `TICKER` table, a definition like `ORDER BY rowtime ASC, price DESC` is valid, but `ORDER BY price, rowtime` or `ORDER BY rowtime DESC, price ASC` is not. + +Define & Measures +----------------- + +`DEFINE` and `MEASURES` keywords have similar functions as `WHERE` and `SELECT` clauses in a simple SQL query. + +Using the `MEASURES` clause you can define what will be included in the output. What exactly will be produced depends also +on the [output mode](#output-mode) setting. + +On the other hand `DEFINE` allows to specify conditions that rows have to fulfill in order to be classified to corresponding [pattern variable](#defining-pattern). +If a condition is not defined for a pattern variable, a default condition will be used, which evaluates to `true` for every row. + +For a more thorough explanation on expressions that you can use in those clauses please have a look at [event stream navigation](#event-stream-navigation). + +Defining pattern +---------------- + +`MATCH_RECOGNIZE` clause allows user to search for patterns in event streams using a powerful and expressive language +that is somewhat similar to widespread regular expression syntax. Every pattern is constructed from building blocks called +pattern variables, to whom operators (quantifiers and other modifiers) can be applied. The whole pattern must be enclosed in +brackets. Example pattern: + +{% highlight sql %} +PATTERN (A B+ C*? D) +{% endhighlight %} + +One may use the following operators: + +* Concatenation - a pattern like (A B) means that between the A B the contiguity is strict. This means there can be no rows that were not mapped to A or B in between +* Quantifiers - modifies the number of rows that can be mapped to pattern variable + * `*` — 0 or more rows + * `+` — 1 or more rows + * `?` — 0 or 1 rows + * `{ n }` — exactly n rows (n > 0) + * `{ n, }` — n or more rows (n ≥ 0) + * `{ n, m }` — between n and m (inclusive) rows (0 ≤ n ≤ m, 0 < m) + * `{ , m }` — between 0 and m (inclusive) rows (m > 0) + +Attention Patterns that can potentially produce empty match are not supported. +Examples of such patterns are: `PATTERN (A*)`, `PATTERN (A? B*)`, `PATTERN (A{0,} B{0,} C*)` etc. + +### Greedy & reluctant quantifiers + +Each quantifier can be either greedy (true by default) or reluctant. The difference is that greedy quantifiers try to match +as many rows as possible, while reluctant as few as possible. To better illustrate the difference one can analyze following example: + +Query with greedy quantifier applied to `B` variable: +{% highlight sql %} +SELECT * +FROM Ticker + MATCH_RECOGNIZE( + PARTITION BY symbol + ORDER BY rowtime + MEASURES + C.price as lastPrice + PATTERN (A B* C) + ONE ROW PER MATCH + AFTER MATCH SKIP PAST LAST ROW + DEFINE + A as A.price > 10 + B as B.price < 15 + C as B.price > 12 + ) +{% endhighlight %} + +For input: + +{% highlight text %} + symbol tax price rowtime +======= ===== ======== ===================== + XYZ 1 10 2018-09-17 10:00:02 + XYZ 2 11 2018-09-17 10:00:03 + XYZ 1 12 2018-09-17 10:00:04 + XYZ 2 13 2018-09-17 10:00:05 + XYZ 1 14 2018-09-17 10:00:06 + XYZ 2 16 2018-09-17 10:00:07 +{% endhighlight %} + +Will produce output: + +{% highlight text %} + symbol lastPrice +======== =========== + XYZ 16 +{% endhighlight %} + +but the same query with just the `B*` modified to `B*?`, which means it should be reluctant quantifier, will produce: + +{% highlight text %} + symbol lastPrice +======== =========== + XYZ 13 +{% endhighlight %} + +Attention It is not possible to use a greedy quantifier for the last +variable for a pattern, thus pattern like `(A B*)` is not allowed. This can be easily worked around by introducing an artificial state +e.g. `C` that will have a negated condition of `B`. So you could use a query like: + +{% highlight sql %} +PATTERN (A B* C) +DEFINE + A as condA() + B as condB() + C as NOT condB() +{% endhighlight %} + +Attention The optional reluctant quantifier (`A??` or `A{0,1}?`) is not supported right now. + +Output mode +----------- + +The output mode describes how many rows should be emitted for every found match. The SQL standard describes two modes: `ALL ROWS PER MATCH` and +`ONE ROW PER MATCH`. + +Currently the only supported output mode is `ONE ROW PER MATCH` that will always produce one output summary row per each found match. +The schema of the output row will be a union of `{partitioning columns} + {measures columns}` in that particular order. + +Example: + +Query: +{% highlight sql %} +SELECT * +FROM Ticker + MATCH_RECOGNIZE( + PARTITION BY symbol + ORDER BY rowtime + MEASURES + FIRST(A.price) as startPrice + LAST(A.price) as topPrice + B.price as lastPrice + PATTERN (A+ B) + ONE ROW PER MATCH + DEFINE + A as A.price > LAST(A.price, 1) OR LAST(A.price, 1) IS NULL, + B as B.price < LAST(A.price) + ) +{% endhighlight %} + +for input: + +{% highlight text %} + symbol tax price. rowtime +======== ===== ======== ===================== + XYZ 1 10 2018-09-17 10:00:02 + XYZ 2 12 2018-09-17 10:00:03 + XYZ 1 13 2018-09-17 10:00:04 + XYZ 2 11 2018-09-17 10:00:05 +{% endhighlight %} + +will produce: + +{% highlight text %} + symbol startPrice topPrice lastPrice +======== ============ ========== =========== + XYZ 10 13 11 +{% endhighlight %} + +Event stream navigation +------------------ + +### Pattern variable reference +Pattern variable reference allows referencing a set of rows mapped to a particular pattern variable in +`DEFINE` or `MEASURE` clauses. For example, expression `A.price` describes a set of rows mapped so far to `A` plus the current row +if we try to match current row to `A`. If an expression in the `DEFINE`/`MEASURES` requires a single row e.g. `A.price > 10`, `A.price` +it selects the last value belonging to the corresponding set. + +If no pattern variable is specified e.g. `SUM(price)` it references the default pattern variable `*` which references all variables in the pattern. +In other words it creates a set of all the rows mapped so far to any variable plus the current row. + +For a more thorough example one can analyze the following pattern and corresponding conditions: + +{% highlight sql %} +PATTERN (A B+) +DEFINE + A as A.price > 10, + B as B.price > A.price AND SUM(price) < 100 AND SUM(B.price) < 80 +{% endhighlight %} + +The following table describes how those conditions are evaluated for each incoming event. That table consists of following columns: + * `no` - a row identifier, to uniquely reference in `{A.price}/{B.price}/{price}` columns + * `{A.price}/{B.price}/{price}` - describes set of rows which are used in the `DEFINE` clause to evaluate conditions. + * `CLASSIFIER` - is a classifier of the current row, which tells which variable was that row mapped to + * `A.price/B.price/SUM(price)/SUM(B.price)` - describes the result of those expression + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
nopriceCLASSIFIER{A.price}{B.price}{price}A.priceB.priceSUM(price)SUM(B.price)
#110-> A#1--10---
#215-> B#1#2#1,#210152515
#320-> B#1#2,#3#1,#2,#310204535
#431-> B#1#2,#3,#4#1,#2,#3,#410317666
#535#1#2,#3,#4,#5#1,#2,#3,#4,#51035111101
+ +### Pattern variable indexing + +Apache Flink allows navigating within events that were mapped to a particular pattern variable with so called logical offsets. This can be expressed +with two corresponding functions: + +
+ + + + + + + + + + + + + + + + + +
Comparison functionsDescription
+ {% highlight text %} +LAST(variable.field, n) +{% endhighlight %} + +

Returns the value of the field from the event that was mapped to the n-th element of the variable, counting from the last element mapped.

+
+ {% highlight text %} +FIRST(variable.field, n) +{% endhighlight %} + +

Returns the value of the field from the event that was mapped to the n-th element of the variable, counting from the first element mapped.

+
+ +Example query +{% highlight sql %} +PATTERN (A B+) +DEFINE + A as A.price > 10, + B as (B.price > LAST(B.price, 1) OR LAST(B.price, 1) IS NULL) AND + (B.price > 2 * LAST(B.price, 2) OR LAST(B.price, 2) IS NULL) +{% endhighlight %} + +will be evaluated as follows: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
priceCLASSIFIERLAST(B.price, 1)LAST(B.price, 2)Comment
10-> A
15-> BnullnullNotice that LAST(A.price, 1) is null, because there is still nothing mapped to B
20-> B15null
31-> B2015
353120Not mapped because 35 < 2 * 20
+ +It might also make sense to use the default pattern variable with logical offsets, which allows indexing within all the rows mapped so far: + +{% highlight sql %} +PATTERN (A B? C) +DEFINE + B as B.price < 20, + C as LAST(price, 1) < C.price +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
priceCLASSIFIERLAST(price, 1)Comment
10-> A
15-> B
20-> C15LAST(price, 1) was evaluated as the price of the row mapped to the B variable
+ +but if the second row did not map to `B` variable then we would have the following results: + + + + + + + + + + + + + + + + + + + + + + + + +
priceCLASSIFIERLAST(price, 1)Comment
10-> A
20-> C10LAST(price, 1) was evaluated as the price of the row mapped to the A variable.
+ +It is also possible to use multiple pattern variable references but all of them must reference the same variable. +This way one can write an expression on multiple columns. +In other words the value of `LAST`/`FIRST` function must be computed in a single row. + +Thus it is possible to use `LAST(A.price * A.tax)`, but an expression like `LAST(A.price * B.tax)` is not allowed. + +After match skip +---------------- + +This clause specifies where to start a new matching procedure after a complete match was found. + +There are five different strategies: +* `SKIP TO NEXT ROW` - continues searching for a new match starting at the next element to the starting element of a match +* `SKIP PAST LAST ROW` - resumes the pattern matching at the next row after the last row of the current match. +* `SKIP TO FIRST variable` - resumes the pattern matching at the first row that is mapped to the pattern variable +* `SKIP TO LAST variable` - resumes the pattern matching at the last row that is mapped to the pattern variable + +This is also a way to specify how many matches a single event can belong to, e.g. with the `SKIP PAST LAST ROW` every event can belong to at most one match. + +To better understand the differences between those strategies one can analyze the following example: + +Query: + +{% highlight sql %} +SELECT * +FROM Ticker + MATCH_RECOGNIZE( + PARTITION BY symbol + ORDER BY rowtime + MEASURES + SUM(A.price) as sumPrice, + FIRST(A.rowtime) as startTime, + LAST(A.rowtime) as endTime + PATTERN (A+ C) + ONE ROW PER MATCH + [AFTER MATCH STRATEGY] + DEFINE + A as SUM(A.price) < 30 + ) +{% endhighlight %} + +For an input: + +{% highlight text %} + symbol tax price rowtime +======== ===== ======= ===================== + XYZ 1 7 2018-09-17 10:00:01 + XYZ 2 9 2018-09-17 10:00:02 + XYZ 1 10 2018-09-17 10:00:03 + XYZ 2 17 2018-09-17 10:00:04 + XYZ 2 14 2018-09-17 10:00:05 +{% endhighlight %} + +Will produce results based on what `AFTER MATCH STRATEGY` was used: + +* AFTER MATCH SKIP PAST LAST ROW + +{% highlight text %} + symbol sumPrice startTime endTime +======== ========== ===================== ===================== + XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:03 + XYZ 17 2018-09-17 10:00:04 2018-09-17 10:00:04 +{% endhighlight %} + +* AFTER MATCH SKIP TO LAST A + +{% highlight text %} + symbol sumPrice startTime endTime +======== ========== ===================== ===================== + XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:03 + XYZ 27 2018-09-17 10:00:03 2018-09-17 10:00:04 + XYZ 17 2018-09-17 10:00:04 2018-09-17 10:00:04 +{% endhighlight %} + +* AFTER MATCH SKIP TO NEXT ROW + +{% highlight text %} + symbol sumPrice startTime endTime +======== ========== ===================== ===================== + XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:03 + XYZ 19 2018-09-17 10:00:02 2018-09-17 10:00:03 + XYZ 27 2018-09-17 10:00:03 2018-09-17 10:00:04 + XYZ 17 2018-09-17 10:00:04 2018-09-17 10:00:04 +{% endhighlight %} + +* AFTER MATCH SKIP TO FIRST A + +This combination will produce a runtime exception, because one would always try to start a new match where the +last one started. This would produce an infinite loop, thus it is prohibited. + +One has to have in mind that in case of the `SKIP TO FIRST/LAST variable` it might be possible that there are no rows mapped to that +variable.(e.g. For pattern `A*`). In such cases a RuntimeException will be thrown as the Standard requires a valid row to continue +matching. + +Known limitations +----------------- + +`MATCH_RECOGNIZE` clause is still an ongoing effort and therefore some SQL Standard features are not supported yet. +The list of such features includes: +* pattern expressions + * pattern groups - this means that e.g. quantifiers can not be applied to a subsequence of the pattern, thus `(A (B C)+)` is not a valid pattern + * alterations - patterns like `PATTERN((A B | C D) E)`, which means that either a subsequence `A B` or `C D` has to be found before looking for the `E` row + * `PERMUTE` operator - which is equivalent to all permutations of variables that it was applied to e.g. `PATTERN (PERMUTE (A, B, C))` = `PATTERN (A B C | A C B | B A C | B C A | C A B | C B A)` + * anchors - `^, $`, which denote beginning/end of a partition, those do not make sense in the streaming context and will not be supported + * exclusion - `PATTERN ({- A -} B)` meaning that `A` will be looked for, but will not participate in the output, this works only for the `ALL ROWS PER MATCH` mode + * reluctant optional quantifier - `PATTERN A??` only the greedy optional quantifier is supported +* `ALL ROWS PER MATCH` output mode, which produces an output row for every row that participated in the creation of a found match. This also means: + * that the only supported semantic for the `MEASURES` clause is `FINAL` + * `CLASSIFIER` function, which returns the pattern variable that a row was mapped to, is not yet supported +* `SUBSET` - which allows creating logical groups of pattern variables and using those groups in the `DEFINE` and `MEASURES` clauses +* physical offsets - `PREV/NEXT`, which indexes all events seen rather than only those that were mapped to a pattern variable(as in [logical offsets](#pattern-variable-indexing) case) +* there is no support of aggregates yet, one cannot use aggregates in `MEASURES` nor `DEFINE` clauses +* user defined functions cannot be used within `MATCH_RECOGNIZE` +* `MATCH_RECOGNIZE` is supported only for SQL, there is no equivalent in the table API + + +### Controlling memory consumption + +Memory consumption is an important aspect when writing `MATCH_RECOGNIZE` queries, as the space of potential matches is built in a breadth first like manner. +Having that in mind one must make sure that the pattern can finish(preferably with a reasonable number of rows mapped to the match as they have to fit into memory) e.g. +it does not have a quantifier without an upper limit that accepts every single row. Such pattern could look like this: + +{% highlight sql %} +PATTERN (A B+ C) +DEFINE + A as A.price > 10, + C as C.price > 20 +{% endhighlight %} + +which will map every incoming row to the `B` variable and thus will never finish. That query could be fixed e.g by negating the condition for `C`: + +{% highlight sql %} +PATTERN (A B+ C) +DEFINE + A as A.price > 10, + B as B.price <= 20, + C as C.price > 20 +{% endhighlight %} + +or using the [reluctant quantifier](#greedy--reluctant-quantifiers): + +{% highlight sql %} +PATTERN (A B+? C) +DEFINE + A as A.price > 10, + C as C.price > 20 +{% endhighlight %} + +One has to be aware that `MATCH_RECOGNIZE` clause does not use [state retention time](query_configuration.html#idle-state-retention-time). There is also no possibility to +define a time restriction on the pattern to finish as of now, because there is no such possibility in SQL standard. The Community is in the process of designing a proper syntax for that +feature right now. diff --git a/docs/dev/table/streaming/query_configuration.md b/docs/dev/table/streaming/query_configuration.md index dc6d11ff6ec17..acfe8e845f1d2 100644 --- a/docs/dev/table/streaming/query_configuration.md +++ b/docs/dev/table/streaming/query_configuration.md @@ -1,7 +1,7 @@ --- title: "Query Configuration" nav-parent_id: streaming_tableapi -nav-pos: 5 +nav-pos: 6 --- -It is a common use-case to search for a set of event patterns, especially in case of data streams. Apache Flink -comes with [CEP library]({{ site.baseurl }}/dev/libs/cep.html) which allows pattern detection in event streams. On the other hand Flink's -SQL API provides a relational way to express queries that comes with multiple functions and -optimizations that can be used out of the box. In December 2016, ISO released a new version of the -international SQL standard ([ISO/IEC 9075:2016](https://standards.iso.org/ittf/PubliclyAvailableStandards/c065143_ISO_IEC_TR_19075-5_2016.zip)) -including the Row Pattern Recognition for complex event processing, which allowes to consolidate those two APIs using `MATCH_RECOGNIZE` clause. +It is a common use-case to search for a set of event patterns, especially in case of data streams. Flink +comes with a [complex event processing (CEP) library]({{ site.baseurl }}/dev/libs/cep.html) which allows for pattern detection in event streams. Furthermore, Flink's +SQL API provides a relational way of expressing queries with a large set of built-in functions and rule-based optimizations that can be used out of the box. -`MATCH_RECOGNIZE` enables you to do the following tasks: +In December 2016, the International Organization for Standardization (ISO) released a new version of the SQL standard which includes _Row Pattern Recognition in SQL_ ([ISO/IEC TR 19075-5:2016](https://standards.iso.org/ittf/PubliclyAvailableStandards/c065143_ISO_IEC_TR_19075-5_2016.zip)). It allows Flink to consolidate CEP and SQL API using the `MATCH_RECOGNIZE` clause for complex event processing in SQL. + +A `MATCH_RECOGNIZE` clause enables the following tasks: * Logically partition and order the data that is used with `PARTITION BY` and `ORDER BY` clauses. -* Define patterns of rows to seek using the `PATTERN` clause. These patterns use a powerful and expressive regular expression syntax. +* Define patterns of rows to seek using the `PATTERN` clause. These patterns use a syntax similar to that of regular expressions. * Specify logical conditions required to map a row to a row pattern variable in the `DEFINE` clause. * Define measures, which are expressions usable in other parts of the SQL query, in the `MEASURES` clause. -Every `MATCH_RECOGNIZE` query consists of the following clauses: +The following example illustrates the syntax for basic pattern recognition: -* [PARTITION BY](#partitioning) - defines logical partitioning of the stream, similar to `GROUP BY` operation. -* [ORDER BY](#order-of-events) - specifies how the incoming events should be ordered, this is essential as patterns depend on order. -* [MEASURES](#define--measures) - defines output of the clause, similar to `SELECT` clause -* [ONE ROW PER MATCH](#output-mode) - output mode which defines how many rows per match should be produced -* [AFTER MATCH SKIP](#after-match-skip) - allows to specify where the next match should start, this is also a way to control how many distinct matches a single event can belong to -* [PATTERN](#defining-pattern) - allows constructing patterns that will be searched for using a regular expression like syntax -* [DEFINE](#define--measures) - this section defines conditions on events that should be met in order to be qualified to the corresponding pattern variable +{% highlight sql %} +SELECT T.aid, T.bid, T.cid +FROM MyTable +MATCH_RECOGNIZE ( + PARTITION BY userid + ORDER BY proctime + MEASURES + A.id AS aid, + B.id AS bid, + C.id AS cid + PATTERN (A B C) + DEFINE + A AS name = 'a', + B AS name = 'b', + C AS name = 'c' +) AS T +{% endhighlight %} +This page will explain each keyword in more detail and will illustrate more complex examples. -This clause can only be applied to an [append](dynamic_tables.html#update-and-append-queries) table and it always produces -an append table as well. +Attention The `MATCH_RECOGNIZE` clause implementation in Flink is just a subset of the bigger pattern recognition standard. Only the features that are documented in the following sections are supported so far. Since the development is still in an early phase, please also take a look at the [known limitations](#known-limitations). * This will be replaced by the TOC {:toc} -Example query -------------- +Introduction and Examples +------------------------- + +### Installation Guide + +The pattern recognition feature uses the Apache Flink's CEP library internally. In order to be able to use the `MATCH_RECOGNIZE` clause, +the library needs to be added as a dependency to your Maven project. + +{% highlight xml %} + + org.apache.flink + flink-cep{{ site.scala_version_suffix }} + {{ site.version }} + +{% endhighlight %} + +Alternatively, you can also add the dependency to the cluster classpath (see the [dependency section]({{ site.baseurl}}/projectsetup/dependencies.html) for more information). + +If you want to use the `MATCH_RECOGNIZE` clause in the [SQL Client]({{ site.baseurl}}/dev/table/sqlClient.html), +you don't have to do anything as all the dependencies are included by default. + +### SQL Semantics + +Every `MATCH_RECOGNIZE` query consists of the following clauses: + +* [PARTITION BY](#partitioning) - defines the logical partitioning of the table; similar to a `GROUP BY` operation. +* [ORDER BY](#order-of-events) - specifies how the incoming rows should be ordered; this is essential as patterns depend on an order. +* [MEASURES](#define--measures) - defines output of the clause; similar to a `SELECT` clause. +* [ONE ROW PER MATCH](#output-mode) - output mode which defines how many rows per match should be produced. +* [AFTER MATCH SKIP](#after-match-strategy) - allows to specify where the next match should start; this is also a way to control how many distinct matches a single event can belong to. +* [PATTERN](#defining-pattern) - allows constructing patterns that will be searched for using a _regular expression_-like syntax. +* [DEFINE](#define--measures) - this section defines conditions on rows that should be met in order to be qualified to the corresponding pattern variable. + +Attention Currently, the `MATCH_RECOGNIZE` clause can only be applied to an [append table](dynamic_tables.html#update-and-append-queries). Furthermore, it always produces +an append table as well. + +### Examples + +For our examples, we assume that a table `Ticker` has been registered. The table contains prices of stocks at a particular point in time. Each row represents an updated characteristic of the ticker. -Having a table `TICKER`, describing prices of stocks in a particular moment of time, each row represents an updated characteristic of a ticker. The table has a following schema: {% highlight text %} @@ -65,13 +110,13 @@ Ticker |-- symbol: Long # symbol of the stock |-- price: Long # price of the stock |-- tax: Long # tax liability of the stock - |-- rowTime: TimeIndicatorTypeInfo(rowtime) # point in time when change to those values happened + |-- rowtime: TimeIndicatorTypeInfo(rowtime) # point in time when the change to those values happened {% endhighlight %} -The incoming data for a single ticker could look following: +For simplification, we only consider the incoming data for a single stock `ACME`. A ticker could look similar to the following table where rows are continuously appended. {% highlight text %} -SYMBOL ROWTIME PRICE TAX +symbol rowtime price tax ====== ==================== ======= ======= 'ACME' '01-Apr-11 10:00:00' 12 1 'ACME' '01-Apr-11 10:00:01' 17 2 @@ -86,7 +131,7 @@ SYMBOL ROWTIME PRICE TAX 'ACME' '01-Apr-11 10:00:10' 19 1 {% endhighlight %} -For example to find periods of a constantly decreasing price of a single ticker one could write a query like this: +The task is now to find periods of a constantly decreasing price of a single ticker. For this, one could write a query like: {% highlight sql %} SELECT * @@ -95,84 +140,87 @@ MATCH_RECOGNIZE ( PARTITION BY symbol ORDER BY rowtime MEASURES - STRT_ROW.rowtime AS start_tstamp, - LAST(PRICE_DOWN.rowtime) AS bottom_tstamp, - LAST(PRICE_UP.rowtime) AS end_tstamp + STRT_ROW.rowtime AS start_tstamp, + LAST(PRICE_DOWN.rowtime) AS bottom_tstamp, + LAST(PRICE_UP.rowtime) AS end_tstamp ONE ROW PER MATCH - AFTER MATCH SKIP TO LAST UP + AFTER MATCH SKIP TO LAST PRICE_UP PATTERN (STRT_ROW PRICE_DOWN+ PRICE_UP) DEFINE - PRICE_DOWN AS PRICE_DOWN.price < LAST(PRICE_DOWN.price, 1) OR - (LAST(PRICE_DOWN.price, 1) IS NULL AND PRICE_DOWN.price < STRT_ROW.price)) - PRICE_UP AS PRICE_UP.price > LAST(PRICE_DOWN.price, 1) + PRICE_DOWN AS + (LAST(PRICE_DOWN.price, 1) IS NULL AND PRICE_DOWN.price < STRT_ROW.price)) OR + PRICE_DOWN.price < LAST(PRICE_DOWN.price, 1) + PRICE_UP AS + PRICE_UP.price > LAST(PRICE_DOWN.price, 1) ) MR; {% endhighlight %} -which will produce a summary row for each found period in which the price was constantly decreasing. +The query partitions the `Ticker` table by the `symbol` column and orders it by the `rowtime` time attribute. -{% highlight text %} -SYMBOL START_TST BOTTOM_TS END_TSTAM -========= ================== ================== ================== -ACME 01-APR-11 10:00:04 01-APR-11 10:00:07 01-APR-11 10:00:08 -{% endhighlight %} +The `PATTERN` clause specifies that we are interested in a pattern with a starting event `STRT_ROW` that is followed by one or more `PRICE_DOWN` events and concluded with a `PRICE_UP` event. If such a pattern can be found, the next pattern match will be seeked at the last `PRICE_UP` event as indicated by the `AFTER MATCH SKIP TO LAST` clause. -The resulting row says that the query found a period of a decreasing price for a ticker that started at `01-APR-11 10:00:04`, -achieved the lowest value at `01-APR-11 10:00:07` and the price increased at `01-APR-11 10:00:08` +The `DEFINE` clause specifies the conditions that need to be met for a `PRICE_DOWN` and `PRICE_UP` event. Although the `STRT_ROW` pattern variable is not present it has an implicit condition that is evaluated always as `TRUE`. -Installation guide ------------------- +A pattern variable `PRICE_DOWN` is defined as a row with a price that is smaller than the price of the last row that met the `PRICE_DOWN` condition. For the initial case or when there is no last row that met the `PRICE_DOWN` condition, the price of the row should be smaller than the price of the preceding row in the pattern (referenced by `START_ROW`). -Pattern recognition feature uses the Apache Flink's CEP library internally. In order to be able to use this clause one has to add -the library as a dependency. Either by adding it to your uber-jar or by adding a dependency on: +A pattern variable `PRICE_UP` is defined as a row with a price that is larger than the price of the last row that met the `PRICE_DOWN` condition. -{% highlight xml %} - - org.apache.flink - flink-cep{{ site.scala_version_suffix }} - {{ site.version }} - +The query produces a summary row for each found period in which the price was constantly decreasing. + +The exact representation of output rows is defined in `MEASURES` part of the query. The number of output rows is defined by the `ONE ROW PER MATCH` output mode. + +{% highlight text %} + symbol start_tstamp bottom_tstamp end_tstamp +========= ================== ================== ================== +ACME 01-APR-11 10:00:04 01-APR-11 10:00:07 01-APR-11 10:00:08 {% endhighlight %} -or by adding it to the cluster classpath (see [here]({{ site.baseurl}}/dev/linking.html)). If you want to use -the `MATCH_RECOGNIZE` clause from [sql-client]({{ site.baseurl}}/dev/table/sqlClient.html) you don't have to do anything as all the dependencies are included by default. +The resulting row contains the found period of a decreasing price for a ticker that started at `01-APR-11 10:00:04` and +achieved the lowest price at `01-APR-11 10:00:07` that increased again at `01-APR-11 10:00:08`. Partitioning ------------ -It is possible to look for patterns in a partitioned data, e.g. trends for a single ticker. This can be expressed using the `PARTITION BY` clause. It is similar to using the `GROUP BY` for aggregations. -Attention It is highly advised to partition the incoming data because otherwise the `MATCH_RECOGNIZE` will be translated +It is possible to look for patterns in partitioned data, e.g., trends for a single ticker or a particular user. This can be expressed using the `PARTITION BY` clause. The clause is similar to using `GROUP BY` for aggregations. + +Attention It is highly advised to partition the incoming data because otherwise the `MATCH_RECOGNIZE` clause will be translated into a non-parallel operator to ensure global ordering. -Order of events +Order of Events --------------- -Apache Flink allows searching for patterns based on time, either [processing-time or event-time](time_attributes.html). This assumption -is very important, because it allows sorting events before they are passed to pattern state machine. This ensures -that the produced output will be correct in regards not to the order the rows arrived, but in which the event corresponding to the row really happened. +Apache Flink allows for searching for patterns based on time; either [processing time or event time](time_attributes.html). -As a consequence one has to provide a time attribute with ascending ordering as the first argument to `ORDER BY` clause. -That means for the example `TICKER` table, a definition like `ORDER BY rowtime ASC, price DESC` is valid, but `ORDER BY price, rowtime` or `ORDER BY rowtime DESC, price ASC` is not. +In case of event time, this assumption is very important because it enables to sort the events before they are passed to the internal pattern state machine. As a consequence, the +produced output will be correct regardless of the order in which rows are appended to the table. Instead, the pattern is evaluated in the order specified by the time contained in each row. + +The `MATCH_RECOGNIZE` clause assumes a [time attribute](time_attributes.html) with ascending ordering as the first argument to `ORDER BY` clause. + +For the example `Ticker` table, a definition like `ORDER BY rowtime ASC, price DESC` is valid but `ORDER BY price, rowtime` or `ORDER BY rowtime DESC, price ASC` is not. Define & Measures ----------------- -`DEFINE` and `MEASURES` keywords have similar functions as `WHERE` and `SELECT` clauses in a simple SQL query. +The `DEFINE` and `MEASURES` keywords have similar meaning as `WHERE` and `SELECT` clauses in a simple SQL query. -Using the `MEASURES` clause you can define what will be included in the output. What exactly will be produced depends also -on the [output mode](#output-mode) setting. +The `MEASURES` clause defines what will be included in the output of a matching pattern. It can project columns and defines expressions for evaluation. +The number of produced rows depends on the [output mode](#output-mode) setting. -On the other hand `DEFINE` allows to specify conditions that rows have to fulfill in order to be classified to corresponding [pattern variable](#defining-pattern). -If a condition is not defined for a pattern variable, a default condition will be used, which evaluates to `true` for every row. +The `DEFINE` clause allows to specify conditions that rows have to fulfill in order to be classified to a corresponding [pattern variable](#defining-pattern). +If a condition is not defined for a pattern variable, a default condition will be used which evaluates to `true` for every row. -For a more thorough explanation on expressions that you can use in those clauses please have a look at [event stream navigation](#event-stream-navigation). +For a more detailed explanation about expressions that can be used in those clauses, please have a look at the [event stream navigation](#pattern-navigation) section. + +Defining a Pattern +------------------ -Defining pattern ----------------- +The `MATCH_RECOGNIZE` clause allows users to search for patterns in event streams using a powerful and expressive syntax +that is somewhat similar to the widespread regular expression syntax. -`MATCH_RECOGNIZE` clause allows user to search for patterns in event streams using a powerful and expressive language -that is somewhat similar to widespread regular expression syntax. Every pattern is constructed from building blocks called -pattern variables, to whom operators (quantifiers and other modifiers) can be applied. The whole pattern must be enclosed in -brackets. Example pattern: +Every pattern is constructed from basic building blocks, called _pattern variables_, to which operators (quantifiers and other modifiers) can be applied. The whole pattern must be enclosed in +brackets. + +An example pattern could look like: {% highlight sql %} PATTERN (A B+ C*? D) @@ -180,25 +228,26 @@ PATTERN (A B+ C*? D) One may use the following operators: -* Concatenation - a pattern like (A B) means that between the A B the contiguity is strict. This means there can be no rows that were not mapped to A or B in between -* Quantifiers - modifies the number of rows that can be mapped to pattern variable - * `*` — 0 or more rows - * `+` — 1 or more rows - * `?` — 0 or 1 rows - * `{ n }` — exactly n rows (n > 0) - * `{ n, }` — n or more rows (n ≥ 0) - * `{ n, m }` — between n and m (inclusive) rows (0 ≤ n ≤ m, 0 < m) - * `{ , m }` — between 0 and m (inclusive) rows (m > 0) +* _Concatenation_ - a pattern like `(A B)` means that the contiguity is strict between `A` and `B`. Therefore, there can be no rows that were not mapped to `A` or `B` in between. +* _Quantifiers_ - modify the number of rows that can be mapped to the pattern variable. + * `*` — _0_ or more rows + * `+` — _1_ or more rows + * `?` — _0_ or _1_ rows + * `{ n }` — exactly _n_ rows (_n > 0_) + * `{ n, }` — _n_ or more rows (_n ≥ 0_) + * `{ n, m }` — between _n_ and _m_ (inclusive) rows (_0 ≤ n ≤ m, 0 < m_) + * `{ , m }` — between _0_ and _m_ (inclusive) rows (_m > 0_) + +Attention Patterns that can potentially produce an empty match are not supported. +Examples of such patterns are `PATTERN (A*)`, `PATTERN (A? B*)`, `PATTERN (A{0,} B{0,} C*)`, etc. -Attention Patterns that can potentially produce empty match are not supported. -Examples of such patterns are: `PATTERN (A*)`, `PATTERN (A? B*)`, `PATTERN (A{0,} B{0,} C*)` etc. +### Greedy & Reluctant Quantifiers -### Greedy & reluctant quantifiers +Each quantifier can be either _greedy_ (default behavior) or _reluctant_. Greedy quantifiers try to match +as many rows as possible while reluctant quantifiers try to match as few as possible. -Each quantifier can be either greedy (true by default) or reluctant. The difference is that greedy quantifiers try to match -as many rows as possible, while reluctant as few as possible. To better illustrate the difference one can analyze following example: +In order to illustrate the difference, one can view the following example with a query where a greedy quantifier is applied to the `B` variable: -Query with greedy quantifier applied to `B` variable: {% highlight sql %} SELECT * FROM Ticker @@ -206,18 +255,18 @@ FROM Ticker PARTITION BY symbol ORDER BY rowtime MEASURES - C.price as lastPrice + C.price AS lastPrice PATTERN (A B* C) ONE ROW PER MATCH AFTER MATCH SKIP PAST LAST ROW DEFINE - A as A.price > 10 - B as B.price < 15 - C as B.price > 12 + A AS A.price > 10 + B AS B.price < 15 + C AS B.price > 12 ) {% endhighlight %} -For input: +Given we have the following input: {% highlight text %} symbol tax price rowtime @@ -230,7 +279,7 @@ For input: XYZ 2 16 2018-09-17 10:00:07 {% endhighlight %} -Will produce output: +The pattern above will produce the following output: {% highlight text %} symbol lastPrice @@ -238,7 +287,7 @@ Will produce output: XYZ 16 {% endhighlight %} -but the same query with just the `B*` modified to `B*?`, which means it should be reluctant quantifier, will produce: +The same query where `B*` is modified to `B*?`, which means that `B*` should be reluctant, will produce: {% highlight text %} symbol lastPrice @@ -246,32 +295,35 @@ but the same query with just the `B*` modified to `B*?`, which means it should b XYZ 13 {% endhighlight %} +The pattern variable `B` matches only to the row with price `12` instead of swallowing the rows with prices `12`, `13`, and `14`. + Attention It is not possible to use a greedy quantifier for the last -variable for a pattern, thus pattern like `(A B*)` is not allowed. This can be easily worked around by introducing an artificial state -e.g. `C` that will have a negated condition of `B`. So you could use a query like: +variable of a pattern. Thus, a pattern like `(A B*)` is not allowed. This can be easily worked around by introducing an artificial state +(e.g. `C`) that has a negated condition of `B`. So you could use a query like: {% highlight sql %} PATTERN (A B* C) DEFINE - A as condA() - B as condB() - C as NOT condB() + A AS condA() + B AS condB() + C AS NOT condB() {% endhighlight %} Attention The optional reluctant quantifier (`A??` or `A{0,1}?`) is not supported right now. -Output mode +Output Mode ----------- -The output mode describes how many rows should be emitted for every found match. The SQL standard describes two modes: `ALL ROWS PER MATCH` and -`ONE ROW PER MATCH`. +The _output mode_ describes how many rows should be emitted for every found match. The SQL standard describes two modes: +- `ALL ROWS PER MATCH` +- `ONE ROW PER MATCH`. -Currently the only supported output mode is `ONE ROW PER MATCH` that will always produce one output summary row per each found match. -The schema of the output row will be a union of `{partitioning columns} + {measures columns}` in that particular order. +Currently, the only supported output mode is `ONE ROW PER MATCH` that will always produce one output summary row for each found match. -Example: +The schema of the output row will be a concatenation of `[partitioning columns] + [measures columns]` in that particular order. + +The following example shows the output of a query defined as: -Query: {% highlight sql %} SELECT * FROM Ticker @@ -279,21 +331,21 @@ FROM Ticker PARTITION BY symbol ORDER BY rowtime MEASURES - FIRST(A.price) as startPrice - LAST(A.price) as topPrice - B.price as lastPrice + FIRST(A.price) AS startPrice + LAST(A.price) AS topPrice + B.price AS lastPrice PATTERN (A+ B) ONE ROW PER MATCH DEFINE - A as A.price > LAST(A.price, 1) OR LAST(A.price, 1) IS NULL, - B as B.price < LAST(A.price) + A AS LAST(A.price, 1) IS NULL OR A.price > LAST(A.price, 1), + B AS B.price < LAST(A.price) ) {% endhighlight %} -for input: +For the following input rows: {% highlight text %} - symbol tax price. rowtime + symbol tax price rowtime ======== ===== ======== ===================== XYZ 1 10 2018-09-17 10:00:02 XYZ 2 12 2018-09-17 10:00:03 @@ -301,7 +353,7 @@ for input: XYZ 2 11 2018-09-17 10:00:05 {% endhighlight %} -will produce: +The query will produce the following output: {% highlight text %} symbol startPrice topPrice lastPrice @@ -309,42 +361,56 @@ will produce: XYZ 10 13 11 {% endhighlight %} -Event stream navigation +The pattern recognition is partitioned by the `symbol` column. Even though not explicitly mentioned in the `MEASURES` clause, the partitioned column is added at the beginning of the result. + +Pattern Navigation ------------------ -### Pattern variable reference -Pattern variable reference allows referencing a set of rows mapped to a particular pattern variable in -`DEFINE` or `MEASURE` clauses. For example, expression `A.price` describes a set of rows mapped so far to `A` plus the current row -if we try to match current row to `A`. If an expression in the `DEFINE`/`MEASURES` requires a single row e.g. `A.price > 10`, `A.price` +The `DEFINE` and `MEASURE` clauses allow for navigating within a list of (potentially) matching rows to a pattern. + +This section discusses this navigation for declaring conditions or producing output results. + +### Pattern Variable Referencing + +A _pattern variable reference_ allows to reference a set of rows mapped to a particular pattern variable in +`DEFINE` or `MEASURE` clauses. + +For example, the expression `A.price` describes a set of rows mapped so far to `A` plus the current row +if we try to match the current row to `A`. If an expression in the `DEFINE`/`MEASURES` clause requires a single row (e.g. `A.price` or `A.price > 10`), it selects the last value belonging to the corresponding set. -If no pattern variable is specified e.g. `SUM(price)` it references the default pattern variable `*` which references all variables in the pattern. -In other words it creates a set of all the rows mapped so far to any variable plus the current row. +If no pattern variable is specified (e.g. `SUM(price)`), an expression references the default pattern variable `*` which references all variables in the pattern. +In other words, it creates a list of all the rows mapped so far to any variable plus the current row. + +#### Example -For a more thorough example one can analyze the following pattern and corresponding conditions: +For a more thorough example, one can take a look at the following pattern and corresponding conditions: {% highlight sql %} PATTERN (A B+) DEFINE - A as A.price > 10, - B as B.price > A.price AND SUM(price) < 100 AND SUM(B.price) < 80 + A AS A.price > 10, + B AS B.price > A.price AND SUM(price) < 100 AND SUM(B.price) < 80 {% endhighlight %} -The following table describes how those conditions are evaluated for each incoming event. That table consists of following columns: - * `no` - a row identifier, to uniquely reference in `{A.price}/{B.price}/{price}` columns - * `{A.price}/{B.price}/{price}` - describes set of rows which are used in the `DEFINE` clause to evaluate conditions. - * `CLASSIFIER` - is a classifier of the current row, which tells which variable was that row mapped to - * `A.price/B.price/SUM(price)/SUM(B.price)` - describes the result of those expression +The following table describes how those conditions are evaluated for each incoming event. + +The table consists of the following columns: + * `#` - the row identifier that uniquely identifies an incoming row in the lists `[A.price]`/`[B.price]`/`[price]`. + * `price` - the price of the incoming row. + * `[A.price]`/`[B.price]`/`[price]` - describe lists of rows which are used in the `DEFINE` clause to evaluate conditions. + * `Classifier` - the classifier of the current row which indicates the pattern variable the row is mapped to. + * `A.price`/`B.price`/`SUM(price)`/`SUM(B.price)` - describes the result after those expressions have been evaluated. - + - - - - + + + + @@ -370,7 +436,7 @@ The following table describes how those conditions are evaluated for each incomi - + @@ -381,8 +447,8 @@ The following table describes how those conditions are evaluated for each incomi - - + + @@ -393,8 +459,8 @@ The following table describes how those conditions are evaluated for each incomi - - + + @@ -405,8 +471,8 @@ The following table describes how those conditions are evaluated for each incomi - - + + @@ -415,59 +481,70 @@ The following table describes how those conditions are evaluated for each incomi
no# priceCLASSIFIER{A.price}{B.price}{price}Classifier[A.price][B.price][price] A.price B.price SUM(price)-> B #1 #2#1,#2#1, #2 10 15 2520 -> B #1#2,#3#1,#2,#3#2, #3#1, #2, #3 10 20 4531 -> B #1#2,#3,#4#1,#2,#3,#4#2, #3, #4#1, #2, #3, #4 10 31 7635 #1#2,#3,#4,#5#1,#2,#3,#4,#5#2, #3, #4, #5#1, #2, #3, #4, #5 10 35 111
-### Pattern variable indexing +As can be seen in the table, the first row is mapped to pattern variable `A` and subsequent rows are mapped to pattern variable `B`. However, the last row does not fulfill the `B` condition because the sum over all mapped rows `SUM(price)` and the sum over all rows in `B` exceed the specified thresholds. + +Attention Please note that aggregations such as `SUM` are not supported yet. They are only used for explanation here. + +### Logical Offsets -Apache Flink allows navigating within events that were mapped to a particular pattern variable with so called logical offsets. This can be expressed +_Logical offsets_ allow to navigate within events that were mapped to a particular pattern variable. This can be expressed with two corresponding functions: -
- - - - - - - - +
Comparison functionsDescription
+ - + + + + + + - - - - + + + + - - - -
- {% highlight text %} + Offset functionsDescription
+ {% highlight text %} LAST(variable.field, n) {% endhighlight %} - -

Returns the value of the field from the event that was mapped to the n-th element of the variable, counting from the last element mapped.

-
- {% highlight text %} + +

Returns the value of the field from the event that was mapped to the n-th last element of the variable. The counting starts at the last element mapped.

+
+ {% highlight text %} FIRST(variable.field, n) {% endhighlight %} - -

Returns the value of the field from the event that was mapped to the n-th element of the variable, counting from the first element mapped.

-
+ + +

Returns the value of the field from the event that was mapped to the n-th element of the variable. The counting starts at the first element mapped.

+ + + + + +#### Examples + +For a more thorough example, one can take a look at the following pattern and corresponding conditions: -Example query {% highlight sql %} PATTERN (A B+) DEFINE - A as A.price > 10, - B as (B.price > LAST(B.price, 1) OR LAST(B.price, 1) IS NULL) AND - (B.price > 2 * LAST(B.price, 2) OR LAST(B.price, 2) IS NULL) + A AS A.price > 10, + B AS (LAST(B.price, 1) IS NULL OR B.price > LAST(B.price, 1)) AND + (LAST(B.price, 2) IS NULL OR B.price > 2 * LAST(B.price, 2)) {% endhighlight %} -will be evaluated as follows: +The following table describes how those conditions are evaluated for each incoming event. + +The table consists of the following columns: + * `price` - the price of the incoming row. + * `Classifier` - the classifier of the current row which indicates the pattern variable the row is mapped to. + * `LAST(B.price, 1)`/`LAST(B.price, 2)` - describes the result after those expressions have been evaluated. - + @@ -486,7 +563,7 @@ will be evaluated as follows: - + @@ -507,25 +584,27 @@ will be evaluated as follows: - +
priceCLASSIFIERClassifier LAST(B.price, 1) LAST(B.price, 2) Comment-> B null nullNotice that LAST(A.price, 1) is null, because there is still nothing mapped to BNotice that LAST(A.price, 1) is null because there is still nothing mapped to B.
20 31 20Not mapped because 35 < 2 * 20Not mapped because 35 < 2 * 20.
-It might also make sense to use the default pattern variable with logical offsets, which allows indexing within all the rows mapped so far: +It might also make sense to use the default pattern variable with logical offsets. + +In this case, an offset considers all the rows mapped so far: {% highlight sql %} PATTERN (A B? C) DEFINE - B as B.price < 20, - C as LAST(price, 1) < C.price + B AS B.price < 20, + C AS LAST(price, 1) < C.price {% endhighlight %} - + @@ -547,18 +626,18 @@ DEFINE - +
priceCLASSIFIERClassifier LAST(price, 1) Comment
20 -> C 15LAST(price, 1) was evaluated as the price of the row mapped to the B variableLAST(price, 1) is evaluated as the price of the row mapped to the B variable.
-but if the second row did not map to `B` variable then we would have the following results: +If the second row did not map to the `B` variable, we would have the following results: - + @@ -574,33 +653,47 @@ but if the second row did not map to `B` variable then we would have the followi - +
priceCLASSIFIERClassifier LAST(price, 1) Comment
20 -> C 10LAST(price, 1) was evaluated as the price of the row mapped to the A variable.LAST(price, 1) is evaluated as the price of the row mapped to the A variable.
-It is also possible to use multiple pattern variable references but all of them must reference the same variable. -This way one can write an expression on multiple columns. -In other words the value of `LAST`/`FIRST` function must be computed in a single row. +It is also possible to use multiple pattern variable references in the first argument of the `FIRST/LAST` functions. This way, one can write an expression that accesses multiple columns. +However, all of them must use the same pattern variable. In other words, the value of the `LAST`/`FIRST` function must be computed in a single row. -Thus it is possible to use `LAST(A.price * A.tax)`, but an expression like `LAST(A.price * B.tax)` is not allowed. +Thus, it is possible to use `LAST(A.price * A.tax)` but an expression like `LAST(A.price * B.tax)` is not allowed. -After match skip ----------------- +After Match Strategy +-------------------- -This clause specifies where to start a new matching procedure after a complete match was found. +The `AFTER MATCH SKIP` clause specifies where to start a new matching procedure after a complete match was found. There are five different strategies: -* `SKIP TO NEXT ROW` - continues searching for a new match starting at the next element to the starting element of a match * `SKIP PAST LAST ROW` - resumes the pattern matching at the next row after the last row of the current match. -* `SKIP TO FIRST variable` - resumes the pattern matching at the first row that is mapped to the pattern variable -* `SKIP TO LAST variable` - resumes the pattern matching at the last row that is mapped to the pattern variable +* `SKIP TO NEXT ROW` - continues searching for a new match starting at the next row after the starting row of the match. +* `SKIP TO LAST variable` - resumes the pattern matching at the last row that is mapped to the specified pattern variable. +* `SKIP TO FIRST variable` - resumes the pattern matching at the first row that is mapped to the specified pattern variable. + +This is also a way to specify how many matches a single event can belong to. For example, with the `SKIP PAST LAST ROW` strategy every event can belong to at most one match. + +#### Examples + +In order to better understand the differences between those strategies one can take a look at the following example. -This is also a way to specify how many matches a single event can belong to, e.g. with the `SKIP PAST LAST ROW` every event can belong to at most one match. +For the following input rows: -To better understand the differences between those strategies one can analyze the following example: +{% highlight text %} + symbol tax price rowtime +======== ===== ======= ===================== + XYZ 1 7 2018-09-17 10:00:01 + XYZ 2 9 2018-09-17 10:00:02 + XYZ 1 10 2018-09-17 10:00:03 + XYZ 2 5 2018-09-17 10:00:04 + XYZ 2 17 2018-09-17 10:00:05 + XYZ 2 14 2018-09-17 10:00:06 +{% endhighlight %} -Query: +We evaluate the following query with different strategies: {% highlight sql %} SELECT * @@ -609,97 +702,93 @@ FROM Ticker PARTITION BY symbol ORDER BY rowtime MEASURES - SUM(A.price) as sumPrice, - FIRST(A.rowtime) as startTime, - LAST(A.rowtime) as endTime + SUM(A.price) AS sumPrice, + FIRST(rowtime) AS startTime, + LAST(rowtime) AS endTime PATTERN (A+ C) ONE ROW PER MATCH [AFTER MATCH STRATEGY] DEFINE - A as SUM(A.price) < 30 + A AS SUM(A.price) < 30 ) {% endhighlight %} -For an input: +The query returns the sum of the prices of all rows mapped to `A` and the first and last timestamp of the overall match. -{% highlight text %} - symbol tax price rowtime -======== ===== ======= ===================== - XYZ 1 7 2018-09-17 10:00:01 - XYZ 2 9 2018-09-17 10:00:02 - XYZ 1 10 2018-09-17 10:00:03 - XYZ 2 17 2018-09-17 10:00:04 - XYZ 2 14 2018-09-17 10:00:05 -{% endhighlight %} +Attention Please note that aggregations such as `SUM` are not supported yet. They are only used for explanation here. -Will produce results based on what `AFTER MATCH STRATEGY` was used: +The query will produce different results based on which `AFTER MATCH` strategy was used: -* AFTER MATCH SKIP PAST LAST ROW +##### `AFTER MATCH SKIP PAST LAST ROW` {% highlight text %} symbol sumPrice startTime endTime ======== ========== ===================== ===================== - XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:03 - XYZ 17 2018-09-17 10:00:04 2018-09-17 10:00:04 + XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:04 + XYZ 17 2018-09-17 10:00:05 2018-09-17 10:00:06 {% endhighlight %} -* AFTER MATCH SKIP TO LAST A +The first result matched against the rows #1, #2, #3, #4. + +The second result matched against the rows #5, #6. + +##### `AFTER MATCH SKIP TO NEXT ROW` {% highlight text %} symbol sumPrice startTime endTime ======== ========== ===================== ===================== - XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:03 - XYZ 27 2018-09-17 10:00:03 2018-09-17 10:00:04 - XYZ 17 2018-09-17 10:00:04 2018-09-17 10:00:04 + XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:04 + XYZ 24 2018-09-17 10:00:02 2018-09-17 10:00:05 + XYZ 15 2018-09-17 10:00:03 2018-09-17 10:00:05 + XYZ 22 2018-09-17 10:00:04 2018-09-17 10:00:06 + XYZ 17 2018-09-17 10:00:05 2018-09-17 10:00:06 {% endhighlight %} -* AFTER MATCH SKIP TO NEXT ROW +Again, the first result matched against the rows #1, #2, #3, #4. + +Compared to the previous strategy, the next match includes row #2 again for the next matching. Therefore, the second result matched against the rows #2, #3, #4, #5. + +The third result matched against the rows #3, #4, #5. + +The forth result matched against the rows #4, #5, #6. + +The last result matched against the rows #5, #6. + +##### `AFTER MATCH SKIP TO LAST A` {% highlight text %} symbol sumPrice startTime endTime ======== ========== ===================== ===================== - XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:03 - XYZ 19 2018-09-17 10:00:02 2018-09-17 10:00:03 - XYZ 27 2018-09-17 10:00:03 2018-09-17 10:00:04 - XYZ 17 2018-09-17 10:00:04 2018-09-17 10:00:04 + XYZ 26 2018-09-17 10:00:01 2018-09-17 10:00:04 + XYZ 15 2018-09-17 10:00:03 2018-09-17 10:00:05 + XYZ 22 2018-09-17 10:00:04 2018-09-17 10:00:06 + XYZ 17 2018-09-17 10:00:05 2018-09-17 10:00:06 {% endhighlight %} -* AFTER MATCH SKIP TO FIRST A +Again, the first result matched against the rows #1, #2, #3, #4. -This combination will produce a runtime exception, because one would always try to start a new match where the -last one started. This would produce an infinite loop, thus it is prohibited. +Compared to the previous strategy, the next match includes only row #3 (mapped to `A`) again for the next matching. Therefore, the second result matched against the rows #3, #4, #5. -One has to have in mind that in case of the `SKIP TO FIRST/LAST variable` it might be possible that there are no rows mapped to that -variable.(e.g. For pattern `A*`). In such cases a RuntimeException will be thrown as the Standard requires a valid row to continue -matching. +The third result matched against the rows #4, #5, #6. -Known limitations ------------------ +The last result matched against the rows #5, #6. -`MATCH_RECOGNIZE` clause is still an ongoing effort and therefore some SQL Standard features are not supported yet. -The list of such features includes: -* pattern expressions - * pattern groups - this means that e.g. quantifiers can not be applied to a subsequence of the pattern, thus `(A (B C)+)` is not a valid pattern - * alterations - patterns like `PATTERN((A B | C D) E)`, which means that either a subsequence `A B` or `C D` has to be found before looking for the `E` row - * `PERMUTE` operator - which is equivalent to all permutations of variables that it was applied to e.g. `PATTERN (PERMUTE (A, B, C))` = `PATTERN (A B C | A C B | B A C | B C A | C A B | C B A)` - * anchors - `^, $`, which denote beginning/end of a partition, those do not make sense in the streaming context and will not be supported - * exclusion - `PATTERN ({- A -} B)` meaning that `A` will be looked for, but will not participate in the output, this works only for the `ALL ROWS PER MATCH` mode - * reluctant optional quantifier - `PATTERN A??` only the greedy optional quantifier is supported -* `ALL ROWS PER MATCH` output mode, which produces an output row for every row that participated in the creation of a found match. This also means: - * that the only supported semantic for the `MEASURES` clause is `FINAL` - * `CLASSIFIER` function, which returns the pattern variable that a row was mapped to, is not yet supported -* `SUBSET` - which allows creating logical groups of pattern variables and using those groups in the `DEFINE` and `MEASURES` clauses -* physical offsets - `PREV/NEXT`, which indexes all events seen rather than only those that were mapped to a pattern variable(as in [logical offsets](#pattern-variable-indexing) case) -* there is no support of aggregates yet, one cannot use aggregates in `MEASURES` nor `DEFINE` clauses -* user defined functions cannot be used within `MATCH_RECOGNIZE` -* `MATCH_RECOGNIZE` is supported only for SQL, there is no equivalent in the table API +##### `AFTER MATCH SKIP TO FIRST A` + +This combination will produce a runtime exception because one would always try to start a new match where the +last one started. This would produce an infinite loop and, thus, is prohibited. + +One has to keep in mind that in case of the `SKIP TO FIRST/LAST variable`strategy it might be possible that there are no rows mapped to that +variable (e.g. for pattern `A*`). In such cases, a runtime exception will be thrown as the standard requires a valid row to continue the +matching. -### Controlling memory consumption +### Controlling Memory Consumption -Memory consumption is an important aspect when writing `MATCH_RECOGNIZE` queries, as the space of potential matches is built in a breadth first like manner. -Having that in mind one must make sure that the pattern can finish(preferably with a reasonable number of rows mapped to the match as they have to fit into memory) e.g. -it does not have a quantifier without an upper limit that accepts every single row. Such pattern could look like this: +Memory consumption is an important aspect when writing `MATCH_RECOGNIZE` queries as the space of potential matches is built in a breadth-first like manner. +Having that in mind, one must make sure that the pattern can finish. Preferably with a reasonable number of rows mapped to the match as they have to fit into memory. + +For example, the pattern must not have a quantifier without an upper limit that accepts every single row. Such a pattern could look like this: {% highlight sql %} PATTERN (A B+ C) @@ -708,7 +797,7 @@ DEFINE C as C.price > 20 {% endhighlight %} -which will map every incoming row to the `B` variable and thus will never finish. That query could be fixed e.g by negating the condition for `C`: +The query will map every incoming row to the `B` variable and thus will never finish. This query could be fixed, e.g., by negating the condition for `C`: {% highlight sql %} PATTERN (A B+ C) @@ -718,7 +807,7 @@ DEFINE C as C.price > 20 {% endhighlight %} -or using the [reluctant quantifier](#greedy--reluctant-quantifiers): +Or by using the [reluctant quantifier](#greedy--reluctant-quantifiers): {% highlight sql %} PATTERN (A B+? C) @@ -727,6 +816,28 @@ DEFINE C as C.price > 20 {% endhighlight %} -One has to be aware that `MATCH_RECOGNIZE` clause does not use [state retention time](query_configuration.html#idle-state-retention-time). There is also no possibility to -define a time restriction on the pattern to finish as of now, because there is no such possibility in SQL standard. The Community is in the process of designing a proper syntax for that +Attention Please note that the `MATCH_RECOGNIZE` clause does not use a configured [state retention time](query_configuration.html#idle-state-retention-time). As of now, there is also no possibility to define a time restriction on the pattern to finish because there is no such possibility in the SQL standard. The community is in the process of designing a proper syntax for that feature right now. + +Known Limitations +----------------- + +`MATCH_RECOGNIZE` clause is still an ongoing effort and therefore some SQL standard features are not supported yet. + +The list of such features includes: +* Pattern expressions: + * Pattern groups - this means that e.g. quantifiers can not be applied to a subsequence of the pattern. Thus, `(A (B C)+)` is not a valid pattern. + * Alterations - patterns like `PATTERN((A B | C D) E)`, which means that either a subsequence `A B` or `C D` has to be found before looking for the `E` row. + * `PERMUTE` operator - which is equivalent to all permutations of variables that it was applied to e.g. `PATTERN (PERMUTE (A, B, C))` = `PATTERN (A B C | A C B | B A C | B C A | C A B | C B A)`. + * Anchors - `^, $`, which denote beginning/end of a partition, those do not make sense in the streaming context and will not be supported. + * Exclusion - `PATTERN ({- A -} B)` meaning that `A` will be looked for but will not participate in the output. This works only for the `ALL ROWS PER MATCH` mode. + * Reluctant optional quantifier - `PATTERN A??` only the greedy optional quantifier is supported. +* `ALL ROWS PER MATCH` output mode - which produces an output row for every row that participated in the creation of a found match. This also means: + * that the only supported semantic for the `MEASURES` clause is `FINAL` + * `CLASSIFIER` function, which returns the pattern variable that a row was mapped to, is not yet supported. +* `SUBSET` - which allows creating logical groups of pattern variables and using those groups in the `DEFINE` and `MEASURES` clauses. +* Physical offsets - `PREV/NEXT`, which indexes all events seen rather than only those that were mapped to a pattern variable(as in [logical offsets](#logical-offsets) case). +* Extracting time attributes - there is no possibility to get a time attribute for subsequent time-based operation yet. +* Aggregates - one cannot use aggregates in `MEASURES` nor `DEFINE` clauses yet. +* User defined functions cannot be used within `MATCH_RECOGNIZE`. +* `MATCH_RECOGNIZE` is supported only for SQL. There is no equivalent in the Table API. From ab86d5d11fe454102fd4b3d0fe92b0b1eb195210 Mon Sep 17 00:00:00 2001 From: David Anderson Date: Fri, 16 Nov 2018 11:49:33 +0100 Subject: [PATCH 118/359] [FLINK-10625] [docs] Polishing MATCH_RECOGNIZE document --- docs/dev/table/sql.md | 2 +- docs/dev/table/streaming/match_recognize.md | 61 ++++++++++----------- 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 2fa742b89f9c9..90e20065726ff 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -958,7 +958,7 @@ val result4 = tableEnv.sqlQuery( Streaming -

Searches for a given pattern in a streaming table according to the MATCH_RECOGNIZE ISO standard. This enables to express complex event processing (CEP) logic in SQL queries.

+

Searches for a given pattern in a streaming table according to the MATCH_RECOGNIZE ISO standard. This makes it possible to express complex event processing (CEP) logic in SQL queries.

For a more detailed description, see the dedicated page for detecting patterns in tables.

{% highlight sql %} diff --git a/docs/dev/table/streaming/match_recognize.md b/docs/dev/table/streaming/match_recognize.md index e10f0f49f469f..b12cbe5e0d927 100644 --- a/docs/dev/table/streaming/match_recognize.md +++ b/docs/dev/table/streaming/match_recognize.md @@ -24,16 +24,16 @@ specific language governing permissions and limitations under the License. --> -It is a common use-case to search for a set of event patterns, especially in case of data streams. Flink +It is a common use case to search for a set of event patterns, especially in case of data streams. Flink comes with a [complex event processing (CEP) library]({{ site.baseurl }}/dev/libs/cep.html) which allows for pattern detection in event streams. Furthermore, Flink's SQL API provides a relational way of expressing queries with a large set of built-in functions and rule-based optimizations that can be used out of the box. In December 2016, the International Organization for Standardization (ISO) released a new version of the SQL standard which includes _Row Pattern Recognition in SQL_ ([ISO/IEC TR 19075-5:2016](https://standards.iso.org/ittf/PubliclyAvailableStandards/c065143_ISO_IEC_TR_19075-5_2016.zip)). It allows Flink to consolidate CEP and SQL API using the `MATCH_RECOGNIZE` clause for complex event processing in SQL. A `MATCH_RECOGNIZE` clause enables the following tasks: -* Logically partition and order the data that is used with `PARTITION BY` and `ORDER BY` clauses. +* Logically partition and order the data that is used with the `PARTITION BY` and `ORDER BY` clauses. * Define patterns of rows to seek using the `PATTERN` clause. These patterns use a syntax similar to that of regular expressions. -* Specify logical conditions required to map a row to a row pattern variable in the `DEFINE` clause. +* The logical components of the row pattern variables are specified in the `DEFINE` clause. * Define measures, which are expressions usable in other parts of the SQL query, in the `MEASURES` clause. The following example illustrates the syntax for basic pattern recognition: @@ -58,7 +58,7 @@ MATCH_RECOGNIZE ( This page will explain each keyword in more detail and will illustrate more complex examples. -Attention The `MATCH_RECOGNIZE` clause implementation in Flink is just a subset of the bigger pattern recognition standard. Only the features that are documented in the following sections are supported so far. Since the development is still in an early phase, please also take a look at the [known limitations](#known-limitations). +Attention Flink's implementation of the `MATCH_RECOGNIZE` clause is a subset of the full standard. Only those features documented in the following sections are supported. Since the development is still in an early phase, please also take a look at the [known limitations](#known-limitations). * This will be replaced by the TOC {:toc} @@ -92,16 +92,16 @@ Every `MATCH_RECOGNIZE` query consists of the following clauses: * [ORDER BY](#order-of-events) - specifies how the incoming rows should be ordered; this is essential as patterns depend on an order. * [MEASURES](#define--measures) - defines output of the clause; similar to a `SELECT` clause. * [ONE ROW PER MATCH](#output-mode) - output mode which defines how many rows per match should be produced. -* [AFTER MATCH SKIP](#after-match-strategy) - allows to specify where the next match should start; this is also a way to control how many distinct matches a single event can belong to. +* [AFTER MATCH SKIP](#after-match-strategy) - specifies where the next match should start; this is also a way to control how many distinct matches a single event can belong to. * [PATTERN](#defining-pattern) - allows constructing patterns that will be searched for using a _regular expression_-like syntax. -* [DEFINE](#define--measures) - this section defines conditions on rows that should be met in order to be qualified to the corresponding pattern variable. +* [DEFINE](#define--measures) - this section defines the conditions that the pattern variables must satisfy. Attention Currently, the `MATCH_RECOGNIZE` clause can only be applied to an [append table](dynamic_tables.html#update-and-append-queries). Furthermore, it always produces an append table as well. ### Examples -For our examples, we assume that a table `Ticker` has been registered. The table contains prices of stocks at a particular point in time. Each row represents an updated characteristic of the ticker. +For our examples, we assume that a table `Ticker` has been registered. The table contains prices of stocks at a particular point in time. The table has a following schema: @@ -140,15 +140,15 @@ MATCH_RECOGNIZE ( PARTITION BY symbol ORDER BY rowtime MEASURES - STRT_ROW.rowtime AS start_tstamp, + START_ROW.rowtime AS start_tstamp, LAST(PRICE_DOWN.rowtime) AS bottom_tstamp, LAST(PRICE_UP.rowtime) AS end_tstamp ONE ROW PER MATCH AFTER MATCH SKIP TO LAST PRICE_UP - PATTERN (STRT_ROW PRICE_DOWN+ PRICE_UP) + PATTERN (START_ROW PRICE_DOWN+ PRICE_UP) DEFINE PRICE_DOWN AS - (LAST(PRICE_DOWN.price, 1) IS NULL AND PRICE_DOWN.price < STRT_ROW.price)) OR + (LAST(PRICE_DOWN.price, 1) IS NULL AND PRICE_DOWN.price < START_ROW.price) OR PRICE_DOWN.price < LAST(PRICE_DOWN.price, 1) PRICE_UP AS PRICE_UP.price > LAST(PRICE_DOWN.price, 1) @@ -157,17 +157,17 @@ MATCH_RECOGNIZE ( The query partitions the `Ticker` table by the `symbol` column and orders it by the `rowtime` time attribute. -The `PATTERN` clause specifies that we are interested in a pattern with a starting event `STRT_ROW` that is followed by one or more `PRICE_DOWN` events and concluded with a `PRICE_UP` event. If such a pattern can be found, the next pattern match will be seeked at the last `PRICE_UP` event as indicated by the `AFTER MATCH SKIP TO LAST` clause. +The `PATTERN` clause specifies that we are interested in a pattern with a starting event `START_ROW` that is followed by one or more `PRICE_DOWN` events and concluded with a `PRICE_UP` event. If such a pattern can be found, the next pattern match will be seeked at the last `PRICE_UP` event as indicated by the `AFTER MATCH SKIP TO LAST` clause. -The `DEFINE` clause specifies the conditions that need to be met for a `PRICE_DOWN` and `PRICE_UP` event. Although the `STRT_ROW` pattern variable is not present it has an implicit condition that is evaluated always as `TRUE`. +The `DEFINE` clause specifies the conditions that need to be met for a `PRICE_DOWN` and `PRICE_UP` event. Although the `START_ROW` pattern variable is not present it has an implicit condition that is evaluated always as `TRUE`. A pattern variable `PRICE_DOWN` is defined as a row with a price that is smaller than the price of the last row that met the `PRICE_DOWN` condition. For the initial case or when there is no last row that met the `PRICE_DOWN` condition, the price of the row should be smaller than the price of the preceding row in the pattern (referenced by `START_ROW`). A pattern variable `PRICE_UP` is defined as a row with a price that is larger than the price of the last row that met the `PRICE_DOWN` condition. -The query produces a summary row for each found period in which the price was constantly decreasing. +This query produces a summary row for each period in which the price of a stock was continuously decreasing. -The exact representation of output rows is defined in `MEASURES` part of the query. The number of output rows is defined by the `ONE ROW PER MATCH` output mode. +The exact representation of the output rows is defined in the `MEASURES` part of the query. The number of output rows is defined by the `ONE ROW PER MATCH` output mode. {% highlight text %} symbol start_tstamp bottom_tstamp end_tstamp @@ -175,7 +175,7 @@ The exact representation of output rows is defined in `MEASURES` part of the que ACME 01-APR-11 10:00:04 01-APR-11 10:00:07 01-APR-11 10:00:08 {% endhighlight %} -The resulting row contains the found period of a decreasing price for a ticker that started at `01-APR-11 10:00:04` and +The resulting row describes a period of falling prices that started at `01-APR-11 10:00:04` and achieved the lowest price at `01-APR-11 10:00:07` that increased again at `01-APR-11 10:00:08`. Partitioning @@ -191,7 +191,7 @@ Order of Events Apache Flink allows for searching for patterns based on time; either [processing time or event time](time_attributes.html). -In case of event time, this assumption is very important because it enables to sort the events before they are passed to the internal pattern state machine. As a consequence, the +In case of event time, the events are sorted before they are passed to the internal pattern state machine. As a consequence, the produced output will be correct regardless of the order in which rows are appended to the table. Instead, the pattern is evaluated in the order specified by the time contained in each row. The `MATCH_RECOGNIZE` clause assumes a [time attribute](time_attributes.html) with ascending ordering as the first argument to `ORDER BY` clause. @@ -201,12 +201,12 @@ For the example `Ticker` table, a definition like `ORDER BY rowtime ASC, price D Define & Measures ----------------- -The `DEFINE` and `MEASURES` keywords have similar meaning as `WHERE` and `SELECT` clauses in a simple SQL query. +The `DEFINE` and `MEASURES` keywords have similar meanings to the `WHERE` and `SELECT` clauses in a simple SQL query. -The `MEASURES` clause defines what will be included in the output of a matching pattern. It can project columns and defines expressions for evaluation. +The `MEASURES` clause defines what will be included in the output of a matching pattern. It can project columns and define expressions for evaluation. The number of produced rows depends on the [output mode](#output-mode) setting. -The `DEFINE` clause allows to specify conditions that rows have to fulfill in order to be classified to a corresponding [pattern variable](#defining-pattern). +The `DEFINE` clause specifies conditions that rows have to fulfill in order to be classified to a corresponding [pattern variable](#defining-pattern). If a condition is not defined for a pattern variable, a default condition will be used which evaluates to `true` for every row. For a more detailed explanation about expressions that can be used in those clauses, please have a look at the [event stream navigation](#pattern-navigation) section. @@ -223,7 +223,7 @@ brackets. An example pattern could look like: {% highlight sql %} -PATTERN (A B+ C*? D) +PATTERN (A B+ C* D) {% endhighlight %} One may use the following operators: @@ -366,14 +366,13 @@ The pattern recognition is partitioned by the `symbol` column. Even though not e Pattern Navigation ------------------ -The `DEFINE` and `MEASURE` clauses allow for navigating within a list of (potentially) matching rows to a pattern. +The `DEFINE` and `MEASURE` clauses allow for navigating within the list of rows that (potentially) match a pattern. This section discusses this navigation for declaring conditions or producing output results. ### Pattern Variable Referencing -A _pattern variable reference_ allows to reference a set of rows mapped to a particular pattern variable in -`DEFINE` or `MEASURE` clauses. +A _pattern variable reference_ allows a set of rows mapped to a particular pattern variable in the `DEFINE` or `MEASURE` clauses to be referenced. For example, the expression `A.price` describes a set of rows mapped so far to `A` plus the current row if we try to match the current row to `A`. If an expression in the `DEFINE`/`MEASURES` clause requires a single row (e.g. `A.price` or `A.price > 10`), @@ -487,7 +486,7 @@ As can be seen in the table, the first row is mapped to pattern variable `A` and ### Logical Offsets -_Logical offsets_ allow to navigate within events that were mapped to a particular pattern variable. This can be expressed +_Logical offsets_ enable navigation within the events that were mapped to a particular pattern variable. This can be expressed with two corresponding functions: @@ -661,14 +660,14 @@ If the second row did not map to the `B` variable, we would have the following r It is also possible to use multiple pattern variable references in the first argument of the `FIRST/LAST` functions. This way, one can write an expression that accesses multiple columns. However, all of them must use the same pattern variable. In other words, the value of the `LAST`/`FIRST` function must be computed in a single row. -Thus, it is possible to use `LAST(A.price * A.tax)` but an expression like `LAST(A.price * B.tax)` is not allowed. +Thus, it is possible to use `LAST(A.price * A.tax)`, but an expression like `LAST(A.price * B.tax)` is not allowed. After Match Strategy -------------------- The `AFTER MATCH SKIP` clause specifies where to start a new matching procedure after a complete match was found. -There are five different strategies: +There are four different strategies: * `SKIP PAST LAST ROW` - resumes the pattern matching at the next row after the last row of the current match. * `SKIP TO NEXT ROW` - continues searching for a new match starting at the next row after the starting row of the match. * `SKIP TO LAST variable` - resumes the pattern matching at the last row that is mapped to the specified pattern variable. @@ -785,7 +784,7 @@ matching. ### Controlling Memory Consumption -Memory consumption is an important aspect when writing `MATCH_RECOGNIZE` queries as the space of potential matches is built in a breadth-first like manner. +Memory consumption is an important consideration when writing `MATCH_RECOGNIZE` queries, as the space of potential matches is built in a breadth-first-like manner. Having that in mind, one must make sure that the pattern can finish. Preferably with a reasonable number of rows mapped to the match as they have to fit into memory. For example, the pattern must not have a quantifier without an upper limit that accepts every single row. Such a pattern could look like this: @@ -822,9 +821,9 @@ feature right now. Known Limitations ----------------- -`MATCH_RECOGNIZE` clause is still an ongoing effort and therefore some SQL standard features are not supported yet. +Flink's implementation of the `MATCH_RECOGNIZE` clause is an ongoing effort, and some features of the SQL standard are not yet supported. -The list of such features includes: +Unsupported features include: * Pattern expressions: * Pattern groups - this means that e.g. quantifiers can not be applied to a subsequence of the pattern. Thus, `(A (B C)+)` is not a valid pattern. * Alterations - patterns like `PATTERN((A B | C D) E)`, which means that either a subsequence `A B` or `C D` has to be found before looking for the `E` row. @@ -837,7 +836,7 @@ The list of such features includes: * `CLASSIFIER` function, which returns the pattern variable that a row was mapped to, is not yet supported. * `SUBSET` - which allows creating logical groups of pattern variables and using those groups in the `DEFINE` and `MEASURES` clauses. * Physical offsets - `PREV/NEXT`, which indexes all events seen rather than only those that were mapped to a pattern variable(as in [logical offsets](#logical-offsets) case). -* Extracting time attributes - there is no possibility to get a time attribute for subsequent time-based operation yet. -* Aggregates - one cannot use aggregates in `MEASURES` nor `DEFINE` clauses yet. +* Extracting time attributes - there is currently no possibility to get a time attribute for subsequent time-based operations. +* Aggregates - one cannot use aggregates in `MEASURES` nor `DEFINE` clauses. * User defined functions cannot be used within `MATCH_RECOGNIZE`. * `MATCH_RECOGNIZE` is supported only for SQL. There is no equivalent in the Table API. From 3c1f14ebecf74e7e014a11c8f790061dd0a79b2d Mon Sep 17 00:00:00 2001 From: Mododo Date: Mon, 19 Nov 2018 10:36:32 +0300 Subject: [PATCH 119/359] [FLINK-10916][streaming] Include duplicated user-specified uid in error message --- .../apache/flink/streaming/api/graph/StreamGraphHasherV2.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java index d7c51bcbdeb16..a87f49fc7b8f2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java @@ -177,7 +177,8 @@ private boolean generateNodeHash( for (byte[] previousHash : hashes.values()) { if (Arrays.equals(previousHash, hash)) { - throw new IllegalArgumentException("Hash collision on user-specified ID. " + + throw new IllegalArgumentException("Hash collision on user-specified ID " + + "\"" + userSpecifiedHash + "\". " + "Most likely cause is a non-unique ID. Please check that all IDs " + "specified via `uid(String)` are unique."); } From 96e790a48ab2c9feba3a440fbb44862bc220ca89 Mon Sep 17 00:00:00 2001 From: kkolman Date: Mon, 19 Nov 2018 11:11:14 +0100 Subject: [PATCH 120/359] [FLINK-10925][py] Check for null when closing socket --- .../flink/python/api/streaming/plan/PythonPlanStreamer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/plan/PythonPlanStreamer.java b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/plan/PythonPlanStreamer.java index d25f3d51ffe71..3e6a068d8aafb 100644 --- a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/plan/PythonPlanStreamer.java +++ b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/streaming/plan/PythonPlanStreamer.java @@ -126,7 +126,9 @@ public void close() { process.destroy(); } finally { try { - server.close(); + if (server != null) { + server.close(); + } } catch (IOException e) { LOG.error("Failed to close socket.", e); } From 76974bd531a8b6c654cc7b7f9f1cce15efc4fe14 Mon Sep 17 00:00:00 2001 From: Xpray Date: Thu, 25 Oct 2018 00:02:13 +0800 Subject: [PATCH 121/359] [FLINK-10670] [table] Fix Correlate codegen error This closes #6923. --- .../flink/table/codegen/CodeGenerator.scala | 14 ++++++------- .../codegen/CollectorCodeGenerator.scala | 11 +++++++--- .../table/plan/nodes/CommonCorrelate.scala | 15 +++---------- .../stream/table/CorrelateITCase.scala | 21 +++++++++++++++++++ 4 files changed, 39 insertions(+), 22 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index b0d37370fee61..fdb0d509554f8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -108,39 +108,39 @@ abstract class CodeGenerator( // set of member statements that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableMemberStatements: mutable.LinkedHashSet[String] = + val reusableMemberStatements: mutable.LinkedHashSet[String] = mutable.LinkedHashSet[String]() // set of constructor statements that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableInitStatements: mutable.LinkedHashSet[String] = + val reusableInitStatements: mutable.LinkedHashSet[String] = mutable.LinkedHashSet[String]() // set of open statements for RichFunction that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableOpenStatements: mutable.LinkedHashSet[String] = + val reusableOpenStatements: mutable.LinkedHashSet[String] = mutable.LinkedHashSet[String]() // set of close statements for RichFunction that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableCloseStatements: mutable.LinkedHashSet[String] = + val reusableCloseStatements: mutable.LinkedHashSet[String] = mutable.LinkedHashSet[String]() // set of statements that will be added only once per record; // code should only update member variables because local variables are not accessible if // the code needs to be split; // we use a LinkedHashSet to keep the insertion order - protected val reusablePerRecordStatements: mutable.LinkedHashSet[String] = + val reusablePerRecordStatements: mutable.LinkedHashSet[String] = mutable.LinkedHashSet[String]() // map of initial input unboxing expressions that will be added only once // (inputTerm, index) -> expr - protected val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = + val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = mutable.Map[(String, Int), GeneratedExpression]() // set of constructor statements that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = + val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = mutable.LinkedHashSet[(String, String)]() /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala index 85d858fb75b99..ddc13c5b45b23 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala @@ -59,13 +59,14 @@ class CollectorCodeGenerator( * valid Java class identifier. * @param bodyCode body code for the collector method * @param collectedType The type information of the element collected by the collector + * @param filterGenerator generator containing context information for the generated body code * @return instance of GeneratedCollector */ def generateTableFunctionCollector( name: String, bodyCode: String, collectedType: TypeInformation[Any], - codeGenerator: CodeGenerator) + filterGenerator: CodeGenerator) : GeneratedCollector = { val className = newName(name) @@ -86,6 +87,10 @@ class CollectorCodeGenerator( s"$input2TypeClass $input2Term" // local variable } + reusableMemberStatements ++= filterGenerator.reusableMemberStatements + reusableInitStatements ++= filterGenerator.reusableInitStatements + reusablePerRecordStatements ++= filterGenerator.reusablePerRecordStatements + val funcCode = j""" |public class $className extends ${classOf[TableFunctionCollector[_]].getCanonicalName} { | @@ -98,7 +103,7 @@ class CollectorCodeGenerator( | | @Override | public void open(${classOf[Configuration].getCanonicalName} parameters) throws Exception { - | ${codeGenerator.reuseOpenCode()} + | ${filterGenerator.reuseOpenCode()} | } | | @Override @@ -113,7 +118,7 @@ class CollectorCodeGenerator( | | @Override | public void close() throws Exception { - | ${codeGenerator.reuseCloseCode()} + | ${filterGenerator.reuseCloseCode()} | } |} |""".stripMargin diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala index 53adf6b5d309e..b9b5a3e7fd56e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.plan.nodes import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexShuttle} +import org.apache.calcite.rex.{RexCall, RexNode} import org.apache.calcite.sql.SemiJoinType import org.apache.flink.api.common.functions.Function import org.apache.flink.api.common.typeinfo.TypeInformation @@ -149,18 +149,9 @@ trait CommonCorrelate { |getCollector().collect(${crossResultExpr.resultTerm}); |""".stripMargin } else { - - // adjust indices of InputRefs to adhere to schema expected by generator - val changeInputRefIndexShuttle = new RexShuttle { - override def visitInputRef(inputRef: RexInputRef): RexNode = { - new RexInputRef(inputSchema.arity + inputRef.getIndex, inputRef.getType) - } - } - // Run generateExpression to add init statements (ScalarFunctions) of condition to generator. - // The generated expression is discarded. - generator.generateExpression(condition.get.accept(changeInputRefIndexShuttle)) - filterGenerator.input1Term = filterGenerator.input2Term + // generating filter expressions might add init statements and member fields + // that need to be combined with the statements of the collector code generator val filterCondition = filterGenerator.generateExpression(condition.get) s""" |${filterGenerator.reuseInputUnboxingCode()} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala index f4b1f8059741d..60e03d4cd2460 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala @@ -284,6 +284,27 @@ class CorrelateITCase extends AbstractTestBase { ) } + @Test + def testTableFunctionCollectorInit(): Unit = { + val t = testData(env).toTable(tEnv).as('a, 'b, 'c) + val func0 = new TableFunc0 + + // this case will generate 'timestamp' member field and 'DateFormatter' + val result = t + .join(func0('c) as('d, 'e)) + .where(dateFormat(currentTimestamp(), "yyyyMMdd") === 'd) + .select('c, 'd, 'e) + .toAppendStream[Row] + + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + assertEquals( + Seq(), + StreamITCase.testResults.sorted + ) + } + private def testData( env: StreamExecutionEnvironment) : DataStream[(Int, Long, String)] = { From 571faf45d77fdc7299a18f43d3cda729be90921b Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 9 Nov 2018 14:58:38 +0100 Subject: [PATCH 122/359] [FLINK-10481][e2e] Added retry logic for building docker image --- flink-end-to-end-tests/test-scripts/common.sh | 19 +++++++++++++++++++ .../test-scripts/test_docker_embedded_job.sh | 13 +++++++++++-- 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 4e6254864c127..275d9c49f4b1e 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -663,3 +663,22 @@ function find_latest_completed_checkpoint { local checkpoint_meta_file=$(ls -d ${checkpoint_root_directory}/chk-[1-9]*/_metadata | sort -Vr | head -n1) echo "$(dirname "${checkpoint_meta_file}")" } + +function retry_times() { + local retriesNumber=$1 + local backoff=$2 + local command=${@:3} + + for (( i = 0; i < ${retriesNumber}; i++ )) + do + if ${command}; then + return 0 + fi + + echo "Command: ${command} failed. Retrying..." + sleep ${backoff} + done + + echo "Command: ${command} failed ${retriesNumber} times." + return 1 +} diff --git a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh index 370ef052a4d4e..2d8aa4f47d415 100755 --- a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh +++ b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh @@ -21,6 +21,8 @@ source "$(dirname "$0")"/common.sh DOCKER_MODULE_DIR=${END_TO_END_DIR}/../flink-container/docker DOCKER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts +DOCKER_IMAGE_BUILD_RETRIES=3 +BUILD_BACKOFF_TIME=5 export FLINK_JOB=org.apache.flink.examples.java.wordcount.WordCount export FLINK_DOCKER_IMAGE_NAME=test_docker_embedded_job @@ -30,12 +32,19 @@ export INPUT_PATH=/data/test/input export OUTPUT_PATH=/data/test/output export FLINK_JOB_ARGUMENTS="--input ${INPUT_PATH}/words --output ${OUTPUT_PATH}/docker_wc_out" -# user inside the container must be able to createto workaround in-container permissions +build_image() { + ./build.sh --from-local-dist --job-jar ${FLINK_DIR}/examples/batch/WordCount.jar --image-name ${FLINK_DOCKER_IMAGE_NAME} +} + +# user inside the container must be able to create files, this is a workaround in-container permissions mkdir -p $OUTPUT_VOLUME chmod 777 $OUTPUT_VOLUME cd "$DOCKER_MODULE_DIR" -./build.sh --from-local-dist --job-jar ${FLINK_DIR}/examples/batch/WordCount.jar --image-name ${FLINK_DOCKER_IMAGE_NAME} +if ! retry_times $DOCKER_IMAGE_BUILD_RETRIES ${BUILD_BACKOFF_TIME} build_image; then + echo "Failed to build docker image. Aborting..." + exit 1 +fi cd "$END_TO_END_DIR" docker-compose -f ${DOCKER_MODULE_DIR}/docker-compose.yml -f ${DOCKER_SCRIPTS}/docker-compose.test.yml up --abort-on-container-exit --exit-code-from job-cluster &> /dev/null From 5d7700721f992013d8a1f8a7a2aaf03f310617a0 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 19 Nov 2018 17:43:39 +0100 Subject: [PATCH 123/359] [FLINK-8997] Added sliding window aggregation to datastream test job --- .../DataStreamAllroundTestJobFactory.java | 34 ++++ .../tests/DataStreamAllroundTestProgram.java | 38 +++- .../apache/flink/streaming/tests/Event.java | 22 ++ .../streaming/tests/SemanticsCheckMapper.java | 2 +- .../tests/SlidingWindowCheckMapper.java | 192 ++++++++++++++++++ 5 files changed, 285 insertions(+), 3 deletions(-) create mode 100644 flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java index 8bd649cc5ebb1..4f69cdb4e0437 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; @@ -38,6 +39,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.tests.artificialstate.ArtificalOperatorStateMapper; @@ -86,6 +88,10 @@ *
  • sequence_generator_source.event_time.clock_progress (long, default - 100): The amount of event time to progress per event generated by the sequence generator.
  • *
  • tumbling_window_operator.num_events (long, default - 20L): The duration of the window, indirectly determined by the target number of events in each window. * Total duration is (sliding_window_operator.num_events) * (sequence_generator_source.event_time.clock_progress).
  • + *
  • test_slide_factor (int, default - 3): test_slide_factor (int, default - 3): how many slides there are in a + * single window (in other words at most how many windows may be opened at time for a given key) The length of + * a window will be calculated as (test_slide_size) * (test_slide_factor)
  • + *
  • test_slide_size (long, default - 250): length of a slide of sliding window in milliseconds. The length of a window will be calculated as (test_slide_size) * (test_slide_factor)
  • * */ public class DataStreamAllroundTestJobFactory { @@ -201,6 +207,14 @@ public class DataStreamAllroundTestJobFactory { .key("tumbling_window_operator.num_events") .defaultValue(20L); + private static final ConfigOption TEST_SLIDE_FACTOR = ConfigOptions + .key("test_slide_factor") + .defaultValue(3); + + private static final ConfigOption TEST_SLIDE_SIZE = ConfigOptions + .key("test_slide_size") + .defaultValue(250L); + public static void setupEnvironment(StreamExecutionEnvironment env, ParameterTool pt) throws Exception { // set checkpointing semantics @@ -455,4 +469,24 @@ static ArtificialStateBuilder createListStateBuilder( listStateGenerator, listStateDescriptor); } + + static SlidingEventTimeWindows createSlidingWindow(ParameterTool pt) { + long slideSize = pt.getLong( + TEST_SLIDE_SIZE.key(), + TEST_SLIDE_SIZE.defaultValue()); + + long slideFactor = pt.getInt( + TEST_SLIDE_FACTOR.key(), + TEST_SLIDE_FACTOR.defaultValue() + ); + + return SlidingEventTimeWindows.of(Time.milliseconds(slideSize * slideFactor), Time.milliseconds(slideSize)); + } + + static FlatMapFunction>, String> createSlidingWindowCheckMapper(ParameterTool pt) { + return new SlidingWindowCheckMapper(pt.getInt( + TEST_SLIDE_FACTOR.key(), + TEST_SLIDE_FACTOR.defaultValue() + )); + } } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java index 3c406c7598d5c..1f2758ddc367b 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.formats.avro.typeutils.AvroSerializer; @@ -36,6 +37,9 @@ import java.util.Arrays; import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.applyTumblingWindows; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createArtificialKeyedStateMapper; @@ -43,6 +47,8 @@ import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createEventSource; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createFailureMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createSemanticsCheckMapper; +import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createSlidingWindow; +import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createSlidingWindowCheckMapper; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createTimestampExtractor; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.isSimulateFailures; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.setupEnvironment; @@ -69,6 +75,8 @@ public class DataStreamAllroundTestProgram { private static final String TIME_WINDOW_OPER_NAME = "TumblingWindowOperator"; private static final String SEMANTICS_CHECK_MAPPER_NAME = "SemanticsCheckMapper"; private static final String FAILURE_MAPPER_NAME = "FailureMapper"; + private static final String SLIDING_WINDOW_CHECK_MAPPER_NAME = "SlidingWindowCheckMapper"; + private static final String SLIDING_WINDOW_AGG_NAME = "SlidingWindowOperator"; public static void main(String[] args) throws Exception { final ParameterTool pt = ParameterTool.fromArgs(args); @@ -153,8 +161,34 @@ public void apply(Integer integer, TimeWindow window, Iterable input, Col eventStream3.keyBy(Event::getKey) .flatMap(createSemanticsCheckMapper(pt)) .name(SEMANTICS_CHECK_MAPPER_NAME) - .uid("0007") - .addSink(new PrintSinkFunction<>()).uid("0008"); + .uid("007") + .addSink(new PrintSinkFunction<>()) + .uid("008"); + + // Check sliding windows aggregations. Output all elements assigned to a window and later on + // check if each event was emitted slide_factor number of times + DataStream>> eventStream4 = eventStream2.keyBy(Event::getKey) + .window(createSlidingWindow(pt)) + .apply(new WindowFunction>, Integer, TimeWindow>() { + private static final long serialVersionUID = 3166250579972849440L; + + @Override + public void apply( + Integer key, TimeWindow window, Iterable input, + Collector>> out) throws Exception { + + out.collect(Tuple2.of(key, StreamSupport.stream(input.spliterator(), false).collect(Collectors.toList()))); + } + }) + .name(SLIDING_WINDOW_AGG_NAME) + .uid("009"); + + eventStream4.keyBy(events-> events.f0) + .flatMap(createSlidingWindowCheckMapper(pt)) + .uid("010") + .name(SLIDING_WINDOW_CHECK_MAPPER_NAME) + .addSink(new PrintSinkFunction<>()) + .uid("011"); env.execute("General purpose test job"); } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/Event.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/Event.java index 8c219ed5863ce..af7a0ad6a6f0d 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/Event.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/Event.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.tests; +import java.util.Objects; + /** * The event type of records used in the {@link DataStreamAllroundTestProgram}. */ @@ -51,6 +53,26 @@ public String getPayload() { return payload; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Event event = (Event) o; + return key == event.key && + eventTime == event.eventTime && + sequenceNumber == event.sequenceNumber && + Objects.equals(payload, event.payload); + } + + @Override + public int hashCode() { + return Objects.hash(key, eventTime, sequenceNumber, payload); + } + @Override public String toString() { return "Event{" + diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java index b67e282334288..780e2aecd3e7b 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java @@ -34,7 +34,7 @@ public class SemanticsCheckMapper extends RichFlatMapFunction { private static final long serialVersionUID = -744070793650644485L; /** This value state tracks the current sequence number per key. */ - private volatile ValueState sequenceValue; + private transient ValueState sequenceValue; /** This defines how semantics are checked for each update. */ private final ValidatorFunction validator; diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java new file mode 100644 index 0000000000000..8d3242efda5d2 --- /dev/null +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java @@ -0,0 +1,192 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ListTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Optional; + +/** + * This mapper validates sliding event time window. It checks each event belongs to appropriate number of consecutive + * windows. + */ +public class SlidingWindowCheckMapper extends RichFlatMapFunction>, String> { + + private static final long serialVersionUID = -744070793650644485L; + + /** This value state tracks previously seen events with the number of windows they appeared in. */ + private transient ValueState>> eventsSeenSoFar; + + private transient ValueState lastSequenceNumber; + + private final int slideFactor; + + SlidingWindowCheckMapper(int slideFactor) { + this.slideFactor = slideFactor; + } + + @Override + public void open(Configuration parameters) { + ValueStateDescriptor>> previousWindowDescriptor = + new ValueStateDescriptor<>("eventsSeenSoFar", + new ListTypeInfo<>(new TupleTypeInfo<>(TypeInformation.of(Event.class), BasicTypeInfo.INT_TYPE_INFO))); + + eventsSeenSoFar = getRuntimeContext().getState(previousWindowDescriptor); + + ValueStateDescriptor lastSequenceNumberDescriptor = + new ValueStateDescriptor<>("lastSequenceNumber", BasicTypeInfo.LONG_TYPE_INFO); + + lastSequenceNumber = getRuntimeContext().getState(lastSequenceNumberDescriptor); + } + + @Override + public void flatMap(Tuple2> value, Collector out) throws Exception { + List> previousWindowValues = Optional.ofNullable(eventsSeenSoFar.value()).orElseGet( + Collections::emptyList); + + List newValues = value.f1; + Optional lastEventInWindow = verifyWindowContiguity(newValues, out); + + Long lastSequenceNumberSeenSoFar = lastSequenceNumber.value(); + List> newWindows = + verifyPreviousOccurences(previousWindowValues, newValues, lastSequenceNumberSeenSoFar, out); + + if (lastEventInWindow.isPresent()) { + updateLastSeenSequenceNumber(lastEventInWindow.get(), lastSequenceNumberSeenSoFar, out); + } + + eventsSeenSoFar.update(newWindows); + } + + private void updateLastSeenSequenceNumber( + Event lastEventInWindow, + Long lastSequenceNumberSeenSoFar, + Collector out) throws IOException { + long lastSequenceNumberInWindow = lastEventInWindow.getSequenceNumber(); + if (lastSequenceNumberSeenSoFar == null || lastSequenceNumberInWindow > lastSequenceNumberSeenSoFar) { + lastSequenceNumber.update(lastSequenceNumberInWindow); + } else if (lastSequenceNumberInWindow < lastSequenceNumberSeenSoFar) { + failWithSequenceNumberDecreased(lastEventInWindow, lastSequenceNumberSeenSoFar, out); + } + } + + private void failWithSequenceNumberDecreased( + Event lastEventInWindow, + Long lastSequenceNumberSeenSoFar, + Collector out) { + out.collect(String.format("Last event in current window (%s) has sequence number lower than seen so far (%d)", + lastEventInWindow, + lastSequenceNumberSeenSoFar)); + } + + /** + * Verifies if all values from previous windows appear in the new one. Returns union of all events seen so far that + * were not seen slideFactor number of times yet. + */ + private List> verifyPreviousOccurences( + List> previousWindowValues, + List newValues, + Long lastSequenceNumberSeenSoFar, + Collector out) { + List> newEventsSeenSoFar = new ArrayList<>(); + List seenEvents = new ArrayList<>(); + + for (Tuple2 windowValue : previousWindowValues) { + if (!newValues.contains(windowValue.f0)) { + failWithEventNotSeenAlertMessage(windowValue, newValues, out); + } else { + seenEvents.add(windowValue.f0); + preserveOrDiscardIfSeenSlideFactorTimes(newEventsSeenSoFar, windowValue); + } + } + + addNotSeenValues(newEventsSeenSoFar, newValues, seenEvents, lastSequenceNumberSeenSoFar, out); + + return newEventsSeenSoFar; + } + + private void addNotSeenValues( + List> newEventsSeenSoFar, + List newValues, + List seenValues, + Long lastSequenceNumberSeenSoFar, + Collector out) { + newValues.stream() + .filter(e -> !seenValues.contains(e)) + .forEach(e -> { + if (lastSequenceNumberSeenSoFar == null || e.getSequenceNumber() > lastSequenceNumberSeenSoFar) { + newEventsSeenSoFar.add(Tuple2.of(e, 1)); + } else { + failWithEventSeenTooManyTimesMessage(e, out); + } + }); + } + + private void failWithEventSeenTooManyTimesMessage(Event e, Collector out) { + out.collect(String.format("Alert: event %s seen more than %d times", e, slideFactor)); + } + + private void preserveOrDiscardIfSeenSlideFactorTimes( + List> newEvenstSeenSoFar, + Tuple2 windowValue) { + int timesSeen = windowValue.f1 + 1; + if (timesSeen != slideFactor) { + newEvenstSeenSoFar.add(Tuple2.of(windowValue.f0, timesSeen)); + } + } + + private void failWithEventNotSeenAlertMessage( + Tuple2 previousWindowValue, + List currentWindowValues, + Collector out) { + out.collect(String.format( + "Alert: event %s did not belong to %d consecutive windows. " + + "Event seen so far %d times.Current window: %s", + previousWindowValue.f0, + slideFactor, + previousWindowValue.f1, + currentWindowValues)); + } + + private Optional verifyWindowContiguity(List newValues, Collector out) { + return newValues.stream() + .sorted(Comparator.comparingLong(Event::getSequenceNumber)) + .reduce((event, event2) -> { + if (event2.getSequenceNumber() - 1 != event.getSequenceNumber()) { + out.collect("Alert: events in window out ouf order!"); + } + + return event2; + }); + } +} From 65ef428fd0ec3ee56071f50f26f3221ade1a26ae Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 19 Nov 2018 21:42:37 +0100 Subject: [PATCH 124/359] [FLINK-10893] [tests] Export S3 credentials properly for test scripts --- flink-end-to-end-tests/test-scripts/common_s3.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3.sh b/flink-end-to-end-tests/test-scripts/common_s3.sh index edb289506ec87..fbe8cadecd611 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3.sh @@ -38,7 +38,11 @@ else echo "Found AWS secret key, running the e2e test." fi -AWS_REGION="${AWS_REGION:-us-east-1}" +# export credentials into environment variables for AWS client +export AWS_REGION="${AWS_REGION:-us-east-1}" +export AWS_ACCESS_KEY_ID="$IT_CASE_S3_ACCESS_KEY" +export AWS_SECRET_ACCESS_KEY="$IT_CASE_S3_SECRET_KEY" + AWS_ACCESS_KEY=$IT_CASE_S3_ACCESS_KEY AWS_SECRET_KEY=$IT_CASE_S3_SECRET_KEY From 8c1bee6007376425fcf9915b79d2c646c0e9f071 Mon Sep 17 00:00:00 2001 From: yanghua Date: Mon, 19 Nov 2018 20:13:29 +0800 Subject: [PATCH 125/359] [FLINK-10922] Refactor the placement of the Flink Kafka connector end to end test module --- flink-dist/src/main/assemblies/bin.xml | 42 ------------------- .../flink-streaming-kafka-test-base}/pom.xml | 6 +-- .../test}/base/CustomWatermarkExtractor.java | 2 +- .../kafka/test}/base/KafkaEvent.java | 2 +- .../kafka/test}/base/KafkaEventSchema.java | 2 +- .../kafka/test}/base/KafkaExampleUtil.java | 2 +- .../test}/base/RollingAdditionMapper.java | 2 +- .../flink-streaming-kafka-test}/pom.xml | 10 ++--- .../streaming/kafka/test}/KafkaExample.java | 12 +++--- .../flink-streaming-kafka010-test}/pom.xml | 10 ++--- .../kafka/test}/Kafka010Example.java | 12 +++--- .../scala/kafka/test}/Kafka010Example.scala | 2 +- .../flink-streaming-kafka011-test}/pom.xml | 10 ++--- .../kafka/test}/Kafka011Example.java | 12 +++--- flink-end-to-end-tests/pom.xml | 4 ++ .../test-scripts/test_streaming_kafka.sh | 2 +- .../test-scripts/test_streaming_kafka010.sh | 2 +- .../test-scripts/test_streaming_kafka011.sh | 2 +- flink-examples/pom.xml | 4 -- 19 files changed, 49 insertions(+), 91 deletions(-) rename {flink-examples/flink-examples-streaming-kafka-base => flink-end-to-end-tests/flink-streaming-kafka-test-base}/pom.xml (88%) rename {flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test}/base/CustomWatermarkExtractor.java (97%) rename {flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test}/base/KafkaEvent.java (97%) rename {flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test}/base/KafkaEventSchema.java (97%) rename {flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test}/base/KafkaExampleUtil.java (97%) rename {flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test}/base/RollingAdditionMapper.java (97%) rename {flink-examples/flink-examples-streaming-kafka => flink-end-to-end-tests/flink-streaming-kafka-test}/pom.xml (88%) rename {flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test}/KafkaExample.java (87%) rename {flink-examples/flink-examples-streaming-kafka-0.10 => flink-end-to-end-tests/flink-streaming-kafka010-test}/pom.xml (92%) rename {flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test}/Kafka010Example.java (86%) rename {flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test}/Kafka010Example.scala (98%) rename {flink-examples/flink-examples-streaming-kafka-0.11 => flink-end-to-end-tests/flink-streaming-kafka011-test}/pom.xml (88%) rename {flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka => flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test}/Kafka011Example.java (86%) diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index fa1e80483415f..89228b39271f8 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -187,48 +187,6 @@ under the License. - - - ../flink-examples/flink-examples-streaming-kafka/target - examples/streaming - 0644 - - *.jar - - - flink-examples-streaming-kafka*.jar - original-*.jar - - - - - - ../flink-examples/flink-examples-streaming-kafka-0.10/target - examples/streaming - 0644 - - *.jar - - - flink-examples-streaming-kafka*.jar - original-*.jar - - - - - - ../flink-examples/flink-examples-streaming-kafka-0.11/target - examples/streaming - 0644 - - *.jar - - - flink-examples-streaming-kafka*.jar - original-*.jar - - - ../flink-libraries/flink-gelly-examples/target diff --git a/flink-examples/flink-examples-streaming-kafka-base/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml similarity index 88% rename from flink-examples/flink-examples-streaming-kafka-base/pom.xml rename to flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml index 3238d68f35490..fa033f5590de4 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml @@ -21,14 +21,14 @@ under the License. xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - flink-examples + flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT 4.0.0 - flink-examples-streaming-kafka-base_${scala.binary.version} - flink-examples-streaming-kafka-base + flink-streaming-kafka-test-base_${scala.binary.version} + flink-streaming-kafka-test-base jar diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/CustomWatermarkExtractor.java similarity index 97% rename from flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java rename to flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/CustomWatermarkExtractor.java index 51de582dc0f5c..8b306ac92972f 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/CustomWatermarkExtractor.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/CustomWatermarkExtractor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka.base; +package org.apache.flink.streaming.kafka.test.base; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEvent.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEvent.java similarity index 97% rename from flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEvent.java rename to flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEvent.java index 7a8f84f8ca8e4..491228104e9b2 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEvent.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEvent.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka.base; +package org.apache.flink.streaming.kafka.test.base; /** * The event type used in the {@link Kafka010Example}. diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEventSchema.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEventSchema.java similarity index 97% rename from flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEventSchema.java rename to flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEventSchema.java index ea9c12b605651..c2cc869e5d90d 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaEventSchema.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaEventSchema.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka.base; +package org.apache.flink.streaming.kafka.test.base; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema; diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java similarity index 97% rename from flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.java rename to flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java index 447dec246484b..eab23e6fcfab9 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/KafkaExampleUtil.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka.base; +package org.apache.flink.streaming.kafka.test.base; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.utils.ParameterTool; diff --git a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/RollingAdditionMapper.java similarity index 97% rename from flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.java rename to flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/RollingAdditionMapper.java index e71f86cefba64..927c3eacfaeb8 100644 --- a/flink-examples/flink-examples-streaming-kafka-base/src/main/java/org/apache/flink/streaming/examples/kafka/base/RollingAdditionMapper.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/RollingAdditionMapper.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka.base; +package org.apache.flink.streaming.kafka.test.base; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ValueState; diff --git a/flink-examples/flink-examples-streaming-kafka/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml similarity index 88% rename from flink-examples/flink-examples-streaming-kafka/pom.xml rename to flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml index b02eaa588684a..d0e0b56a1f15e 100644 --- a/flink-examples/flink-examples-streaming-kafka/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml @@ -21,19 +21,19 @@ under the License. xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - flink-examples + flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT 4.0.0 - flink-examples-streaming-kafka_${scala.binary.version} - flink-examples-streaming-kafka + flink-streaming-kafka-test_${scala.binary.version} + flink-streaming-kafka-test org.apache.flink - flink-examples-streaming-kafka-base_${scala.binary.version} + flink-streaming-kafka-test-base_${scala.binary.version} ${project.version} @@ -75,7 +75,7 @@ under the License. false - org.apache.flink.streaming.examples.kafka.KafkaExample + org.apache.flink.streaming.kafka.test.KafkaExample KafkaExample diff --git a/flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java b/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java similarity index 87% rename from flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java rename to flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java index 27e73d18c0817..70885402b2abd 100644 --- a/flink-examples/flink-examples-streaming-kafka/src/main/java/org/apache/flink/streaming/examples/kafka/KafkaExample.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka; +package org.apache.flink.streaming.kafka.test; 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.kafka.FlinkKafkaConsumer; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; -import org.apache.flink.streaming.examples.kafka.base.CustomWatermarkExtractor; -import org.apache.flink.streaming.examples.kafka.base.KafkaEvent; -import org.apache.flink.streaming.examples.kafka.base.KafkaEventSchema; -import org.apache.flink.streaming.examples.kafka.base.KafkaExampleUtil; -import org.apache.flink.streaming.examples.kafka.base.RollingAdditionMapper; +import org.apache.flink.streaming.kafka.test.base.CustomWatermarkExtractor; +import org.apache.flink.streaming.kafka.test.base.KafkaEvent; +import org.apache.flink.streaming.kafka.test.base.KafkaEventSchema; +import org.apache.flink.streaming.kafka.test.base.KafkaExampleUtil; +import org.apache.flink.streaming.kafka.test.base.RollingAdditionMapper; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; /** diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml similarity index 92% rename from flink-examples/flink-examples-streaming-kafka-0.10/pom.xml rename to flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml index fa09fc22dec75..1be584b5ddf2b 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.10/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml @@ -21,19 +21,19 @@ under the License. xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - flink-examples + flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT 4.0.0 - flink-examples-streaming-kafka-0.10_${scala.binary.version} - flink-examples-streaming-kafka-0.10 + flink-streaming-kafka010-test_${scala.binary.version} + flink-streaming-kafka010-test org.apache.flink - flink-examples-streaming-kafka-base_${scala.binary.version} + flink-streaming-kafka-test-base_${scala.binary.version} ${project.version} @@ -119,7 +119,7 @@ under the License. false - org.apache.flink.streaming.examples.kafka.Kafka010Example + org.apache.flink.streaming.kafka.test.Kafka010Example Kafka010Example diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java similarity index 86% rename from flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java rename to flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java index 2df1f5d84963a..0b97179425861 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java +++ b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka; +package org.apache.flink.streaming.kafka.test; 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.kafka.FlinkKafkaConsumer010; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; -import org.apache.flink.streaming.examples.kafka.base.CustomWatermarkExtractor; -import org.apache.flink.streaming.examples.kafka.base.KafkaEvent; -import org.apache.flink.streaming.examples.kafka.base.KafkaEventSchema; -import org.apache.flink.streaming.examples.kafka.base.KafkaExampleUtil; -import org.apache.flink.streaming.examples.kafka.base.RollingAdditionMapper; +import org.apache.flink.streaming.kafka.test.base.CustomWatermarkExtractor; +import org.apache.flink.streaming.kafka.test.base.KafkaEvent; +import org.apache.flink.streaming.kafka.test.base.KafkaEventSchema; +import org.apache.flink.streaming.kafka.test.base.KafkaExampleUtil; +import org.apache.flink.streaming.kafka.test.base.RollingAdditionMapper; /** * A simple example that shows how to read from and write to Kafka. This will read String messages diff --git a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala similarity index 98% rename from flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala rename to flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala index 9f4fdc4c29489..a35faf53d9d4f 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.10/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala +++ b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.scala.examples.kafka +package org.apache.flink.streaming.scala.kafka.test import org.apache.flink.api.common.restartstrategy.RestartStrategies import org.apache.flink.api.common.serialization.SimpleStringSchema diff --git a/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml similarity index 88% rename from flink-examples/flink-examples-streaming-kafka-0.11/pom.xml rename to flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml index c044767a22186..1ff4b945afcf6 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.11/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml @@ -21,20 +21,20 @@ under the License. xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - flink-examples + flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT 4.0.0 - flink-examples-streaming-kafka-0.11_${scala.binary.version} - flink-examples-streaming-kafka-0.11 + flink-streaming-kafka011-test_${scala.binary.version} + flink-streaming-kafka011-test jar org.apache.flink - flink-examples-streaming-kafka-base_${scala.binary.version} + flink-streaming-kafka-test-base_${scala.binary.version} ${project.version} @@ -76,7 +76,7 @@ under the License. false - org.apache.flink.streaming.examples.kafka.Kafka011Example + org.apache.flink.streaming.kafka.test.Kafka011Example Kafka011Example diff --git a/flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.java b/flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java similarity index 86% rename from flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.java rename to flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java index 17b5d011f289b..1f877c5a5b840 100644 --- a/flink-examples/flink-examples-streaming-kafka-0.11/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka011Example.java +++ b/flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.flink.streaming.examples.kafka; +package org.apache.flink.streaming.kafka.test; 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.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011; -import org.apache.flink.streaming.examples.kafka.base.CustomWatermarkExtractor; -import org.apache.flink.streaming.examples.kafka.base.KafkaEvent; -import org.apache.flink.streaming.examples.kafka.base.KafkaEventSchema; -import org.apache.flink.streaming.examples.kafka.base.KafkaExampleUtil; -import org.apache.flink.streaming.examples.kafka.base.RollingAdditionMapper; +import org.apache.flink.streaming.kafka.test.base.CustomWatermarkExtractor; +import org.apache.flink.streaming.kafka.test.base.KafkaEvent; +import org.apache.flink.streaming.kafka.test.base.KafkaEventSchema; +import org.apache.flink.streaming.kafka.test.base.KafkaExampleUtil; +import org.apache.flink.streaming.kafka.test.base.RollingAdditionMapper; /** * A simple example that shows how to read from and write to Kafka. This will read String messages diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 90c32832af1cc..c7e010e4e10e0 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -61,6 +61,10 @@ under the License. flink-metrics-availability-test flink-metrics-reporter-prometheus-test flink-heavy-deployment-stress-test + flink-streaming-kafka-test-base + flink-streaming-kafka-test + flink-streaming-kafka011-test + flink-streaming-kafka010-test diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh index ff36cf1d77808..045bb0933f8e4 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh @@ -22,4 +22,4 @@ set -Eeuo pipefail source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/kafka-common.sh 2.0.1 5.0.0 5.0 -source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/KafkaExample.jar +source "$(dirname "$0")"/test_streaming_kafka_common.sh ${END_TO_END_DIR}/flink-streaming-kafka-test/target/KafkaExample.jar diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh index a55b0bc33c68b..97a71f9005262 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh @@ -22,5 +22,5 @@ set -Eeuo pipefail source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/kafka-common.sh 0.10.2.0 3.2.0 3.2 -source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/Kafka010Example.jar +source "$(dirname "$0")"/test_streaming_kafka_common.sh ${END_TO_END_DIR}/flink-streaming-kafka010-test/target/Kafka010Example.jar diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh index 2e6e6d93debfc..84fa6edc90eca 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh @@ -20,5 +20,5 @@ source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/kafka-common.sh 0.11.0.2 3.2.0 3.2 -source "$(dirname "$0")"/test_streaming_kafka_common.sh $FLINK_DIR/examples/streaming/Kafka011Example.jar +source "$(dirname "$0")"/test_streaming_kafka_common.sh ${END_TO_END_DIR}/flink-streaming-kafka011-test/target/Kafka011Example.jar diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 9846128efa23f..6a9b7e4379315 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -35,11 +35,7 @@ under the License. flink-examples-batch flink-examples-streaming - flink-examples-streaming-kafka-base - flink-examples-streaming-kafka - flink-examples-streaming-kafka-0.10 flink-examples-table - flink-examples-streaming-kafka-0.11 From b992afa2ac43a0602e5bd8353da63d1539aab207 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 19 Nov 2018 16:57:12 +0100 Subject: [PATCH 126/359] [FLINK-10922] Remove Scala Kafka010Example from flink-end-to-end-tests This commit removes the Scala based Kafka010Example from flink-end-to-end-tests/ flink-streaming-kafka010-test module. Moreover, it adds relative paths to their parent pom's and cleans up the flink-streaming-kafka*/pom.xml. --- .../flink-streaming-kafka-test-base/pom.xml | 1 + .../flink-streaming-kafka-test/pom.xml | 7 +- .../flink-streaming-kafka010-test/pom.xml | 51 +---------- .../scala/kafka/test/Kafka010Example.scala | 88 ------------------- .../flink-streaming-kafka011-test/pom.xml | 7 +- 5 files changed, 4 insertions(+), 150 deletions(-) delete mode 100644 flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml index fa033f5590de4..71366483a41ef 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml @@ -24,6 +24,7 @@ under the License. flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT + .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml index d0e0b56a1f15e..a19881e5439ac 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml @@ -24,6 +24,7 @@ under the License. flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT + .. 4.0.0 @@ -48,12 +49,6 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} - - - org.apache.flink - flink-java - ${project.version} - diff --git a/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml index 1be584b5ddf2b..53e1abaf14dd3 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml @@ -24,6 +24,7 @@ under the License. flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT + .. 4.0.0 @@ -48,60 +49,10 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} - - - org.apache.flink - flink-streaming-scala_${scala.binary.version} - ${project.version} - - - - org.apache.flink - flink-java - ${project.version} - - - - - net.alchim31.maven - scala-maven-plugin - - - - scala-compile-first - process-resources - - add-source - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - - - org.scalastyle - scalastyle-maven-plugin - - ${project.basedir}/../../tools/maven/scalastyle-config.xml - - - org.apache.maven.plugins diff --git a/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala deleted file mode 100644 index a35faf53d9d4f..0000000000000 --- a/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/scala/org/apache/flink/streaming/scala/kafka/test/Kafka010Example.scala +++ /dev/null @@ -1,88 +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.streaming.scala.kafka.test - -import org.apache.flink.api.common.restartstrategy.RestartStrategies -import org.apache.flink.api.common.serialization.SimpleStringSchema -import org.apache.flink.api.java.utils.ParameterTool -import org.apache.flink.streaming.api.scala._ -import org.apache.flink.streaming.connectors.kafka.{FlinkKafkaConsumer010, FlinkKafkaProducer010} - -/** - * An example that shows how to read from and write to Kafka. This will read String messages - * from the input topic, prefix them by a configured prefix and output to the output topic. - * - * Please pass the following arguments to run the example: - * {{{ - * --input-topic test-input - * --output-topic test-output - * --bootstrap.servers localhost:9092 - * --zookeeper.connect localhost:2181 - * --group.id myconsumer - * }}} - */ -object Kafka010Example { - - def main(args: Array[String]): Unit = { - - // parse input arguments - val params = ParameterTool.fromArgs(args) - - if (params.getNumberOfParameters < 4) { - println("Missing parameters!\n" - + "Usage: Kafka --input-topic --output-topic " - + "--bootstrap.servers " - + "--zookeeper.connect --group.id [--prefix ]") - return - } - - val prefix = params.get("prefix", "PREFIX:") - - - val env = StreamExecutionEnvironment.getExecutionEnvironment - env.getConfig.disableSysoutLogging - env.getConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(4, 10000)) - // create a checkpoint every 5 seconds - env.enableCheckpointing(5000) - // make parameters available in the web interface - env.getConfig.setGlobalJobParameters(params) - - // create a Kafka streaming source consumer for Kafka 0.10.x - val kafkaConsumer = new FlinkKafkaConsumer010( - params.getRequired("input-topic"), - new SimpleStringSchema, - params.getProperties) - - val messageStream = env - .addSource(kafkaConsumer) - .map(in => prefix + in) - - // create a Kafka producer for Kafka 0.10.x - val kafkaProducer = new FlinkKafkaProducer010( - params.getRequired("output-topic"), - new SimpleStringSchema, - params.getProperties) - - // write data into Kafka - messageStream.addSink(kafkaProducer) - - env.execute("Kafka 0.10 Example") - } - -} diff --git a/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml index 1ff4b945afcf6..577d3a4b2c35b 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml @@ -24,6 +24,7 @@ under the License. flink-end-to-end-tests org.apache.flink 1.7-SNAPSHOT + .. 4.0.0 @@ -49,12 +50,6 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} - - - org.apache.flink - flink-java - ${project.version} - From 84cad5bda061787125cff60df7bcaf53b039c9c1 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 19 Nov 2018 21:03:13 +0100 Subject: [PATCH 127/359] [FLINK-10922] Exclude transitive kafka-clients dependency from base flink-connector-kafka In order to satisfy dependency convergence we need to exclude the kafka-clients from the base flink-connector-kafka-x dependency in every flink-connector-kafka-y module. --- flink-connectors/flink-connector-kafka-0.10/pom.xml | 4 ++++ flink-connectors/flink-connector-kafka-0.11/pom.xml | 4 ++++ flink-connectors/flink-connector-kafka-0.9/pom.xml | 4 ++++ 3 files changed, 12 insertions(+) diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 8e2f338fbbde4..52c283e017ba3 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -53,6 +53,10 @@ under the License. org.apache.kafka kafka_${scala.binary.version} + + org.apache.kafka + kafka-clients + diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml index d8ae959376489..2c91a05863ddc 100644 --- a/flink-connectors/flink-connector-kafka-0.11/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml @@ -53,6 +53,10 @@ under the License. org.apache.kafka kafka_${scala.binary.version} + + org.apache.kafka + kafka-clients + diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index d68a2e68d60f4..197ce2d5f0721 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -60,6 +60,10 @@ under the License. org.apache.kafka kafka_${scala.binary.version} + + org.apache.kafka + kafka-clients + From 43aa594e9f0787e9e1c5fcddaa60e845a748a237 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 19 Nov 2018 15:56:34 +0100 Subject: [PATCH 128/359] [FLINK-10763] [streaming] Fix interval join return type in Scala This commit fixes the wrong result type of interval joins in the DataStream Scala API. The API did not call the Scala type extraction stack but was using the default one offered by the DataStream Java API. This closes #7141. --- .../streaming/api/scala/KeyedStream.scala | 9 ++- .../api/scala/IntervalJoinITCase.scala | 60 +++++++++++-------- 2 files changed, 41 insertions(+), 28 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala index 580dd46d68342..f85e1b538e2c8 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala @@ -201,7 +201,12 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T] * @return Returns a DataStream */ @PublicEvolving - def process[OUT](processJoinFunction: ProcessJoinFunction[IN1, IN2, OUT]): DataStream[OUT] = { + def process[OUT: TypeInformation]( + processJoinFunction: ProcessJoinFunction[IN1, IN2, OUT]) + : DataStream[OUT] = { + + val outType: TypeInformation[OUT] = implicitly[TypeInformation[OUT]] + val javaJoined = new KeyedJavaStream.IntervalJoined[IN1, IN2, KEY]( firstStream.javaStream.asInstanceOf[KeyedJavaStream[IN1, KEY]], secondStream.javaStream.asInstanceOf[KeyedJavaStream[IN2, KEY]], @@ -209,7 +214,7 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T] upperBound, lowerBoundInclusive, upperBoundInclusive) - asScalaStream(javaJoined.process(processJoinFunction)) + asScalaStream(javaJoined.process(processJoinFunction, outType)) } } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala index 80701c6725a8c..a9ace36cc4960 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/IntervalJoinITCase.scala @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.scala +import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction import org.apache.flink.streaming.api.functions.sink.SinkFunction @@ -25,7 +26,8 @@ import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExt import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.util.Collector -import org.junit.{Assert, Test} +import org.junit.Assert.assertTrue +import org.junit.Test import scala.collection.mutable.ListBuffer @@ -47,22 +49,25 @@ class IntervalJoinITCase extends AbstractTestBase { val sink = new ResultSink() - dataStream1.intervalJoin(dataStream2) + val join = dataStream1.intervalJoin(dataStream2) .between(Time.milliseconds(0), Time.milliseconds(2)) - .process(new CombineToStringJoinFunction()) - .addSink(sink) + .process(new CombineJoinFunction()) + + assertTrue(join.dataType.isInstanceOf[CaseClassTypeInfo[_]]) + + join.addSink(sink) env.execute() sink.expectInAnyOrder( - "(key,0):(key,0)", - "(key,0):(key,1)", - "(key,0):(key,2)", + "(key:key,0)", + "(key:key,1)", + "(key:key,2)", - "(key,1):(key,1)", - "(key,1):(key,2)", + "(key:key,2)", + "(key:key,3)", - "(key,2):(key,2)" + "(key:key,4)" ) } @@ -82,18 +87,21 @@ class IntervalJoinITCase extends AbstractTestBase { val sink = new ResultSink() - dataStream1.intervalJoin(dataStream2) + val join = dataStream1.intervalJoin(dataStream2) .between(Time.milliseconds(0), Time.milliseconds(2)) .lowerBoundExclusive() .upperBoundExclusive() - .process(new CombineToStringJoinFunction()) - .addSink(sink) + .process(new CombineJoinFunction()) + + assertTrue(join.dataType.isInstanceOf[CaseClassTypeInfo[_]]) + + join.addSink(sink) env.execute() sink.expectInAnyOrder( - "(key,0):(key,1)", - "(key,1):(key,2)" + "(key:key,1)", + "(key:key,3)" ) } } @@ -102,14 +110,14 @@ object Companion { val results: ListBuffer[String] = new ListBuffer() } -class ResultSink extends SinkFunction[String] { +class ResultSink extends SinkFunction[(String, Long)] { - override def invoke(value: String, context: SinkFunction.Context[_]): Unit = { - Companion.results.append(value) + override def invoke(value: (String, Long), context: SinkFunction.Context[_]): Unit = { + Companion.results.append(value.toString()) } def expectInAnyOrder(expected: String*): Unit = { - Assert.assertTrue(expected.toSet.equals(Companion.results.toSet)) + assertTrue(expected.toSet.equals(Companion.results.toSet)) } } @@ -117,14 +125,14 @@ class TimestampExtractor extends AscendingTimestampExtractor[(String, Long)] { override def extractAscendingTimestamp(element: (String, Long)): Long = element._2 } -class CombineToStringJoinFunction - extends ProcessJoinFunction[(String, Long), (String, Long), String] { +class CombineJoinFunction + extends ProcessJoinFunction[(String, Long), (String, Long), (String, Long)] { override def processElement( - left: (String, Long), - right: (String, Long), - ctx: ProcessJoinFunction[(String, Long), (String, Long), String]#Context, - out: Collector[String]): Unit = { - out.collect(left + ":" + right) + left: (String, Long), + right: (String, Long), + ctx: ProcessJoinFunction[(String, Long), (String, Long), (String, Long)]#Context, + out: Collector[(String, Long)]): Unit = { + out.collect((left._1 + ":" + right._1, left._2 + right._2)) } } From 55e91245492fb2e7a69c5e13133b863ac82699ae Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Thu, 22 Nov 2018 13:36:22 +0800 Subject: [PATCH 129/359] [FLINK-10983][queryable state] Increase port range for NonHAQueryableStateFsBackendITCase This closes #7159 --- .../itcases/NonHAQueryableStateFsBackendITCase.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java index 9116304ebf5a5..36637c0d34200 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java @@ -44,6 +44,7 @@ public class NonHAQueryableStateFsBackendITCase extends AbstractQueryableStateTe // we always use all TaskManagers so that the JM oracle is always properly re-registered private static final int NUM_TMS = 2; private static final int NUM_SLOTS_PER_TM = 2; + private static final int NUM_PORT_COUNT = 100; private static final int QS_PROXY_PORT_RANGE_START = 9084; private static final int QS_SERVER_PORT_RANGE_START = 9089; @@ -86,10 +87,10 @@ private static Configuration getConfig() { config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1); config.setString( QueryableStateOptions.PROXY_PORT_RANGE, - QS_PROXY_PORT_RANGE_START + "-" + (QS_PROXY_PORT_RANGE_START + NUM_TMS)); + QS_PROXY_PORT_RANGE_START + "-" + (QS_PROXY_PORT_RANGE_START + NUM_PORT_COUNT)); config.setString( QueryableStateOptions.SERVER_PORT_RANGE, - QS_SERVER_PORT_RANGE_START + "-" + (QS_SERVER_PORT_RANGE_START + NUM_TMS)); + QS_SERVER_PORT_RANGE_START + "-" + (QS_SERVER_PORT_RANGE_START + NUM_PORT_COUNT)); config.setBoolean(WebOptions.SUBMIT_ENABLE, false); return config; } From d2f2afb9546fc71ed8d675e04b8445738f1c3ae6 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 9 Nov 2018 11:19:53 +0100 Subject: [PATCH 130/359] [hotfix] [table] Refactor aggregation harness tests to be found --- ...wHarnessTest.scala => GroupAggregateHarnessTest.scala} | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/{NonWindowHarnessTest.scala => GroupAggregateHarnessTest.scala} (96%) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala similarity index 96% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala index 7c4f54303289a..8001d79f8fa2b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala @@ -24,20 +24,18 @@ import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator import org.apache.flink.streaming.runtime.streamrecord.StreamRecord -import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.harness.HarnessTestBase._ import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row import org.junit.Test -class NonWindowHarnessTest extends HarnessTestBase { +class GroupAggregateHarnessTest extends HarnessTestBase { protected var queryConfig = new TestStreamQueryConfig(Time.seconds(2), Time.seconds(3)) @Test - def testNonWindow(): Unit = { + def testAggregate(): Unit = { val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new GroupAggProcessFunction( @@ -97,7 +95,7 @@ class NonWindowHarnessTest extends HarnessTestBase { } @Test - def testNonWindowWithRetract(): Unit = { + def testAggregateWithRetract(): Unit = { val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new GroupAggProcessFunction( From ba7ced2094d99c22381ab39b421def5126ee6c23 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 20 Nov 2018 10:47:05 +0100 Subject: [PATCH 131/359] [hotfix] [table] Show deserialization error cause --- .../main/java/org/apache/flink/table/utils/EncodingUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/utils/EncodingUtils.java b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/utils/EncodingUtils.java index 47aac25e89746..5531082611d36 100644 --- a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/utils/EncodingUtils.java +++ b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/utils/EncodingUtils.java @@ -76,7 +76,7 @@ public static T decodeStringToObject(String base64Strin return instance; } catch (Exception e) { throw new ValidationException( - "Unable to deserialize string '" + base64String + "' of base class '" + baseClass.getName() + "'."); + "Unable to deserialize string '" + base64String + "' of base class '" + baseClass.getName() + "'.", e); } } From 3cb24911b9a3fcc081493df735ed1b02a15de3f9 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 20 Nov 2018 10:47:29 +0100 Subject: [PATCH 132/359] [FLINK-10674] [table] Fix handling of retractions after clean up Because state clean up happens in processing time, it might be the case that retractions are arriving after the state has been cleaned up. Before these changes, a new accumulator was created and invalid retraction messages were emitted. This change drops retraction messages for which no accumulator exists. It also refactors the tests and adds more test cases and explanations. This closes #7147. --- .../aggregate/GroupAggProcessFunction.scala | 7 + .../harness/GroupAggregateHarnessTest.scala | 147 ++++++++--- .../runtime/harness/HarnessTestBase.scala | 229 +++++++++++++++--- .../table/runtime/stream/sql/SqlITCase.scala | 37 +++ 4 files changed, 363 insertions(+), 57 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index 397032003ec81..c59efe250c3d2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -95,6 +95,13 @@ class GroupAggProcessFunction( var inputCnt = cntState.value() if (null == accumulators) { + // Don't create a new accumulator for a retraction message. This + // might happen if the retraction message is the first message for the + // key or after a state clean up. + if (!inputC.change) { + return + } + // first accumulate message firstRow = true accumulators = function.createAccumulators() } else { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala index 8001d79f8fa2b..1dce9946877ff 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/GroupAggregateHarnessTest.scala @@ -52,43 +52,42 @@ class GroupAggregateHarnessTest extends HarnessTestBase { testHarness.open() + val expectedOutput = new ConcurrentLinkedQueue[Object]() + // register cleanup timer with 3001 testHarness.setProcessingTime(1) testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1: JInt), 1)) testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, "bbb"), 1)) + expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1: JInt), 1)) // reuse timer 3001 testHarness.setProcessingTime(1000) testHarness.processElement(new StreamRecord(CRow(3L: JLong, 2: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(3L: JLong, 3: JInt), 1)) testHarness.processElement(new StreamRecord(CRow(4L: JLong, 3: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(4L: JLong, 6: JInt), 1)) // register cleanup timer with 4002 testHarness.setProcessingTime(1002) testHarness.processElement(new StreamRecord(CRow(5L: JLong, 4: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(5L: JLong, 10: JInt), 1)) testHarness.processElement(new StreamRecord(CRow(6L: JLong, 2: JInt, "bbb"), 1)) + expectedOutput.add(new StreamRecord(CRow(6L: JLong, 3: JInt), 1)) // trigger cleanup timer and register cleanup timer with 7003 testHarness.setProcessingTime(4003) testHarness.processElement(new StreamRecord(CRow(7L: JLong, 5: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(7L: JLong, 5: JInt), 1)) testHarness.processElement(new StreamRecord(CRow(8L: JLong, 6: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(8L: JLong, 11: JInt), 1)) testHarness.processElement(new StreamRecord(CRow(9L: JLong, 7: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(9L: JLong, 18: JInt), 1)) testHarness.processElement(new StreamRecord(CRow(10L: JLong, 3: JInt, "bbb"), 1)) + expectedOutput.add(new StreamRecord(CRow(10L: JLong, 3: JInt), 1)) val result = testHarness.getOutput - val expectedOutput = new ConcurrentLinkedQueue[Object]() - - expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(3L: JLong, 3: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(4L: JLong, 6: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(5L: JLong, 10: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(6L: JLong, 3: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(7L: JLong, 5: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(8L: JLong, 11: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(9L: JLong, 18: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(10L: JLong, 3: JInt), 1)) - verify(expectedOutput, result) testHarness.close() @@ -112,42 +111,136 @@ class GroupAggregateHarnessTest extends HarnessTestBase { testHarness.open() + val expectedOutput = new ConcurrentLinkedQueue[Object]() + // register cleanup timer with 3001 testHarness.setProcessingTime(1) + // accumulate testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, "aaa"), 1)) + expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1: JInt), 1)) + + // accumulate testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, "bbb"), 2)) + expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1: JInt), 2)) + + // retract for insertion testHarness.processElement(new StreamRecord(CRow(3L: JLong, 2: JInt, "aaa"), 3)) + expectedOutput.add(new StreamRecord(CRow(false, 3L: JLong, 1: JInt), 3)) + expectedOutput.add(new StreamRecord(CRow(3L: JLong, 3: JInt), 3)) + + // retract for deletion + testHarness.processElement(new StreamRecord(CRow(false, 3L: JLong, 2: JInt, "aaa"), 3)) + expectedOutput.add(new StreamRecord(CRow(false, 3L: JLong, 3: JInt), 3)) + expectedOutput.add(new StreamRecord(CRow(3L: JLong, 1: JInt), 3)) + + // accumulate testHarness.processElement(new StreamRecord(CRow(4L: JLong, 3: JInt, "ccc"), 4)) + expectedOutput.add(new StreamRecord(CRow(4L: JLong, 3: JInt), 4)) // trigger cleanup timer and register cleanup timer with 6002 testHarness.setProcessingTime(3002) - testHarness.processElement(new StreamRecord(CRow(5L: JLong, 4: JInt, "aaa"), 5)) - testHarness.processElement(new StreamRecord(CRow(6L: JLong, 2: JInt, "bbb"), 6)) - testHarness.processElement(new StreamRecord(CRow(7L: JLong, 5: JInt, "aaa"), 7)) - testHarness.processElement(new StreamRecord(CRow(8L: JLong, 6: JInt, "eee"), 8)) - testHarness.processElement(new StreamRecord(CRow(9L: JLong, 7: JInt, "aaa"), 9)) - testHarness.processElement(new StreamRecord(CRow(10L: JLong, 3: JInt, "bbb"), 10)) - - val result = testHarness.getOutput - val expectedOutput = new ConcurrentLinkedQueue[Object]() + // retract after clean up + testHarness.processElement(new StreamRecord(CRow(false, 4L: JLong, 3: JInt, "ccc"), 4)) - expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1: JInt), 1)) - expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1: JInt), 2)) - expectedOutput.add(new StreamRecord(CRow(false, 3L: JLong, 1: JInt), 3)) - expectedOutput.add(new StreamRecord(CRow(3L: JLong, 3: JInt), 3)) - expectedOutput.add(new StreamRecord(CRow(4L: JLong, 3: JInt), 4)) + // accumulate + testHarness.processElement(new StreamRecord(CRow(5L: JLong, 4: JInt, "aaa"), 5)) expectedOutput.add(new StreamRecord(CRow(5L: JLong, 4: JInt), 5)) + testHarness.processElement(new StreamRecord(CRow(6L: JLong, 2: JInt, "bbb"), 6)) expectedOutput.add(new StreamRecord(CRow(6L: JLong, 2: JInt), 6)) + + // retract + testHarness.processElement(new StreamRecord(CRow(7L: JLong, 5: JInt, "aaa"), 7)) expectedOutput.add(new StreamRecord(CRow(false, 7L: JLong, 4: JInt), 7)) expectedOutput.add(new StreamRecord(CRow(7L: JLong, 9: JInt), 7)) + + // accumulate + testHarness.processElement(new StreamRecord(CRow(8L: JLong, 6: JInt, "eee"), 8)) expectedOutput.add(new StreamRecord(CRow(8L: JLong, 6: JInt), 8)) + + // retract + testHarness.processElement(new StreamRecord(CRow(9L: JLong, 7: JInt, "aaa"), 9)) expectedOutput.add(new StreamRecord(CRow(false, 9L: JLong, 9: JInt), 9)) expectedOutput.add(new StreamRecord(CRow(9L: JLong, 16: JInt), 9)) + testHarness.processElement(new StreamRecord(CRow(10L: JLong, 3: JInt, "bbb"), 10)) expectedOutput.add(new StreamRecord(CRow(false, 10L: JLong, 2: JInt), 10)) expectedOutput.add(new StreamRecord(CRow(10L: JLong, 5: JInt), 10)) + val result = testHarness.getOutput + + verify(expectedOutput, result) + + testHarness.close() + } + + @Test + def testDistinctAggregateWithRetract(): Unit = { + + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( + new GroupAggProcessFunction( + genDistinctCountAggFunction, + distinctCountAggregationStateType, + true, + queryConfig)) + + val testHarness = + createHarnessTester( + processFunction, + new TupleRowKeySelector[String](2), + BasicTypeInfo.STRING_TYPE_INFO) + + testHarness.open() + + val expectedOutput = new ConcurrentLinkedQueue[Object]() + + // register cleanup timer with 3001 + testHarness.setProcessingTime(1) + + // insert + testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, "aaa"))) + expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1L: JLong))) + testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, "bbb"))) + expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1L: JLong))) + + // distinct count retract then accumulate for downstream operators + testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, "bbb"))) + expectedOutput.add(new StreamRecord(CRow(false, 2L: JLong, 1L: JLong))) + expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1L: JLong))) + + // update count for accumulate + testHarness.processElement(new StreamRecord(CRow(1L: JLong, 2: JInt, "aaa"))) + expectedOutput.add(new StreamRecord(CRow(false, 1L: JLong, 1L: JLong))) + expectedOutput.add(new StreamRecord(CRow(1L: JLong, 2L: JLong))) + + // update count for retraction + testHarness.processElement(new StreamRecord(CRow(false, 1L: JLong, 2: JInt, "aaa"))) + expectedOutput.add(new StreamRecord(CRow(false, 1L: JLong, 2L: JLong))) + expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1L: JLong))) + + // insert + testHarness.processElement(new StreamRecord(CRow(4L: JLong, 3: JInt, "ccc"))) + expectedOutput.add(new StreamRecord(CRow(4L: JLong, 1L: JLong))) + + // retract entirely + testHarness.processElement(new StreamRecord(CRow(false, 4L: JLong, 3: JInt, "ccc"))) + expectedOutput.add(new StreamRecord(CRow(false, 4L: JLong, 1L: JLong))) + + // trigger cleanup timer and register cleanup timer with 6002 + testHarness.setProcessingTime(3002) + + // insert + testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, "aaa"))) + expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1L: JLong))) + + // trigger cleanup timer and register cleanup timer with 9002 + testHarness.setProcessingTime(6002) + + // retract after cleanup + testHarness.processElement(new StreamRecord(CRow(false, 1L: JLong, 1: JInt, "aaa"))) + + val result = testHarness.getOutput + verify(expectedOutput, result) testHarness.close() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala index f70d991e50bd0..e5cceecc560b5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.runtime.harness import java.util.{Comparator, Queue => JQueue} +import org.apache.flink.api.common.state.{MapStateDescriptor, StateDescriptor} import org.apache.flink.api.common.time.Time -import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO} +import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{LONG_TYPE_INFO, STRING_TYPE_INFO} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.functions.KeySelector import org.apache.flink.api.java.typeutils.RowTypeInfo @@ -28,11 +29,11 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil} -import org.apache.flink.table.api.StreamQueryConfig +import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.codegen.GeneratedAggregationsFunction -import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction} -import org.apache.flink.table.functions.aggfunctions.{IntSumWithRetractAggFunction, LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction} +import org.apache.flink.table.functions.aggfunctions.{CountAggFunction, IntSumWithRetractAggFunction, LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction} import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.getAccumulatorTypeOfAggregateFunction +import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction} import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, RowResultSortComparatorWithWatermarks} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.table.utils.EncodingUtils @@ -55,20 +56,16 @@ class HarnessTestBase { val intSumWithRetractAggFunction: String = EncodingUtils.encodeObjectToString(new IntSumWithRetractAggFunction) + val distinctCountAggFunction: String = + EncodingUtils.encodeObjectToString(new CountAggFunction()) + protected val MinMaxRowType = new RowTypeInfo(Array[TypeInformation[_]]( LONG_TYPE_INFO, STRING_TYPE_INFO, LONG_TYPE_INFO), Array("rowtime", "a", "b")) - protected val SumRowType = new RowTypeInfo(Array[TypeInformation[_]]( - LONG_TYPE_INFO, - INT_TYPE_INFO, - STRING_TYPE_INFO), - Array("a", "b", "c")) - protected val minMaxCRowType = new CRowTypeInfo(MinMaxRowType) - protected val sumCRowType = new CRowTypeInfo(SumRowType) protected val minMaxAggregates: Array[AggregateFunction[_, _]] = Array(new LongMinWithRetractAggFunction, @@ -77,15 +74,28 @@ class HarnessTestBase { protected val sumAggregates: Array[AggregateFunction[_, _]] = Array(new IntSumWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]] + protected val distinctCountAggregates: Array[AggregateFunction[_, _]] = + Array(new CountAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]] + protected val minMaxAggregationStateType: RowTypeInfo = new RowTypeInfo(minMaxAggregates.map(getAccumulatorTypeOfAggregateFunction(_)): _*) protected val sumAggregationStateType: RowTypeInfo = new RowTypeInfo(sumAggregates.map(getAccumulatorTypeOfAggregateFunction(_)): _*) + protected val distinctCountAggregationStateType: RowTypeInfo = + new RowTypeInfo(distinctCountAggregates.map(getAccumulatorTypeOfAggregateFunction(_)): _*) + + protected val distinctCountDescriptor: String = EncodingUtils.encodeObjectToString( + new MapStateDescriptor("distinctAgg0", distinctCountAggregationStateType, Types.LONG)) + + protected val minMaxFuncName = "MinMaxAggregateHelper" + protected val sumFuncName = "SumAggregationHelper" + protected val distinctCountFuncName = "DistinctCountAggregationHelper" + val minMaxCode: String = s""" - |public class MinMaxAggregateHelper + |public class $minMaxFuncName | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { | | transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction @@ -94,7 +104,7 @@ class HarnessTestBase { | transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction | fmax = null; | - | public MinMaxAggregateHelper() throws Exception { + | public $minMaxFuncName() throws Exception { | | fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction) | ${classOf[EncodingUtils].getCanonicalName}.decodeStringToObject( @@ -207,25 +217,25 @@ class HarnessTestBase { val sumAggCode: String = s""" - |public final class SumAggregationHelper + |public final class $sumFuncName | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { | | - |transient org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction - |sum = null; - |private final org.apache.flink.table.runtime.aggregate.SingleElementIterable accIt0 = | new org.apache.flink.table.runtime.aggregate.SingleElementIterable(); | - | public SumAggregationHelper() throws Exception { + | public $sumFuncName() throws Exception { | - |sum = (org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction) - |${classOf[EncodingUtils].getCanonicalName}.decodeStringToObject( - | "$intSumWithRetractAggFunction", - | ${classOf[UserDefinedFunction].getCanonicalName}.class); - |} + | sum = (org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction) + | ${classOf[EncodingUtils].getCanonicalName}.decodeStringToObject( + | "$intSumWithRetractAggFunction", + | ${classOf[UserDefinedFunction].getCanonicalName}.class); + | } | | public final void setAggregationResults( | org.apache.flink.types.Row accs, @@ -256,6 +266,12 @@ class HarnessTestBase { | public final void retract( | org.apache.flink.types.Row accs, | org.apache.flink.types.Row input) { + | + | sum.retract( + | ((org.apache.flink.table.functions.aggfunctions.SumWithRetractAccumulator) accs + | .getField + | (0)), + | (java.lang.Integer) input.getField(1)); | } | | public final org.apache.flink.types.Row createAccumulators() @@ -281,6 +297,162 @@ class HarnessTestBase { | input.getField(0)); | } | + | public final org.apache.flink.types.Row createOutputRow() { + | return new org.apache.flink.types.Row(2); + | } + | + | + | public final org.apache.flink.types.Row mergeAccumulatorsPair( + | org.apache.flink.types.Row a, + | org.apache.flink.types.Row b) + | { + | + | return a; + | + | } + | + | public final void resetAccumulator( + | org.apache.flink.types.Row accs) { + | } + | + | public void open(org.apache.flink.api.common.functions.RuntimeContext ctx) { + | } + | + | public void cleanup() { + | } + | + | public void close() { + | } + |} + |""".stripMargin + + val distinctCountAggCode: String = + s""" + |public final class $distinctCountFuncName + | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { + | + | final org.apache.flink.table.functions.aggfunctions.CountAggFunction count; + | + | final org.apache.flink.table.api.dataview.MapView acc0_distinctValueMap_dataview; + | + | final java.lang.reflect.Field distinctValueMap = + | org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField( + | org.apache.flink.table.functions.aggfunctions.DistinctAccumulator.class, + | "distinctValueMap"); + | + | + | private final org.apache.flink.table.runtime.aggregate.SingleElementIterable accIt0 = + | new org.apache.flink.table.runtime.aggregate.SingleElementIterable(); + | + | public $distinctCountFuncName() throws Exception { + | + | count = (org.apache.flink.table.functions.aggfunctions.CountAggFunction) + | ${classOf[EncodingUtils].getCanonicalName}.decodeStringToObject( + | "$distinctCountAggFunction", + | ${classOf[UserDefinedFunction].getCanonicalName}.class); + | + | distinctValueMap.setAccessible(true); + | } + | + | public void open(org.apache.flink.api.common.functions.RuntimeContext ctx) { + | org.apache.flink.api.common.state.StateDescriptor acc0_distinctValueMap_dataview_desc = + | (org.apache.flink.api.common.state.StateDescriptor) + | ${classOf[EncodingUtils].getCanonicalName}.decodeStringToObject( + | "$distinctCountDescriptor", + | ${classOf[StateDescriptor[_, _]].getCanonicalName}.class, + | ctx.getUserCodeClassLoader()); + | acc0_distinctValueMap_dataview = new org.apache.flink.table.dataview.StateMapView( + | ctx.getMapState((org.apache.flink.api.common.state.MapStateDescriptor) + | acc0_distinctValueMap_dataview_desc)); + | } + | + | public final void setAggregationResults( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row output) { + | + | org.apache.flink.table.functions.AggregateFunction baseClass0 = + | (org.apache.flink.table.functions.AggregateFunction) + | count; + | + | org.apache.flink.table.functions.aggfunctions.DistinctAccumulator distinctAcc0 = + | (org.apache.flink.table.functions.aggfunctions.DistinctAccumulator) accs.getField(0); + | org.apache.flink.table.functions.aggfunctions.CountAccumulator acc0 = + | (org.apache.flink.table.functions.aggfunctions.CountAccumulator) + | distinctAcc0.getRealAcc(); + | + | output.setField(1, baseClass0.getValue(acc0)); + | } + | + | public final void accumulate( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row input) throws Exception { + | + | org.apache.flink.table.functions.aggfunctions.DistinctAccumulator distinctAcc0 = + | (org.apache.flink.table.functions.aggfunctions.DistinctAccumulator) accs.getField(0); + | + | distinctValueMap.set(distinctAcc0, acc0_distinctValueMap_dataview); + | + | if (distinctAcc0.add( + | org.apache.flink.types.Row.of((java.lang.Integer) input.getField(1)))) { + | org.apache.flink.table.functions.aggfunctions.CountAccumulator acc0 = + | (org.apache.flink.table.functions.aggfunctions.CountAccumulator) + | distinctAcc0.getRealAcc(); + | + | + | count.accumulate(acc0, (java.lang.Integer) input.getField(1)); + | } + | } + | + | public final void retract( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row input) throws Exception { + | + | org.apache.flink.table.functions.aggfunctions.DistinctAccumulator distinctAcc0 = + | (org.apache.flink.table.functions.aggfunctions.DistinctAccumulator) accs.getField(0); + | + | distinctValueMap.set(distinctAcc0, acc0_distinctValueMap_dataview); + | + | if (distinctAcc0.remove( + | org.apache.flink.types.Row.of((java.lang.Integer) input.getField(1)))) { + | org.apache.flink.table.functions.aggfunctions.CountAccumulator acc0 = + | (org.apache.flink.table.functions.aggfunctions.CountAccumulator) + | distinctAcc0.getRealAcc(); + | + | count.retract(acc0 , (java.lang.Integer) input.getField(1)); + | } + | } + | + | public final org.apache.flink.types.Row createAccumulators() + | { + | + | org.apache.flink.types.Row accs = new org.apache.flink.types.Row(1); + | + | org.apache.flink.table.functions.aggfunctions.CountAccumulator acc0 = + | (org.apache.flink.table.functions.aggfunctions.CountAccumulator) + | count.createAccumulator(); + | org.apache.flink.table.functions.aggfunctions.DistinctAccumulator distinctAcc0 = + | (org.apache.flink.table.functions.aggfunctions.DistinctAccumulator) + | new org.apache.flink.table.functions.aggfunctions.DistinctAccumulator (acc0); + | accs.setField( + | 0, + | distinctAcc0); + | + | return accs; + | } + | + | public final void setForwardedFields( + | org.apache.flink.types.Row input, + | org.apache.flink.types.Row output) + | { + | + | output.setField( + | 0, + | input.getField(0)); + | } + | | public final void setConstantFlags(org.apache.flink.types.Row output) | { | @@ -304,10 +476,8 @@ class HarnessTestBase { | org.apache.flink.types.Row accs) { | } | - | public void open(org.apache.flink.api.common.functions.RuntimeContext ctx) { - | } - | | public void cleanup() { + | acc0_distinctValueMap_dataview.clear(); | } | | public void close() { @@ -315,12 +485,11 @@ class HarnessTestBase { |} |""".stripMargin - - protected val minMaxFuncName = "MinMaxAggregateHelper" - protected val sumFuncName = "SumAggregationHelper" - protected val genMinMaxAggFunction = GeneratedAggregationsFunction(minMaxFuncName, minMaxCode) protected val genSumAggFunction = GeneratedAggregationsFunction(sumFuncName, sumAggCode) + protected val genDistinctCountAggFunction = GeneratedAggregationsFunction( + distinctCountFuncName, + distinctCountAggCode) def createHarnessTester[IN, OUT, KEY]( operator: OneInputStreamOperator[IN, OUT], diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index c3da65f887a94..46dde8e022502 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -263,6 +263,43 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) } + @Test + def testDistinctWithRetraction(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 1L, "Hi")) + data.+=((1, 1L, "Hi World")) + data.+=((1, 1L, "Test")) + data.+=((2, 1L, "Hi World")) + data.+=((2, 1L, "Test")) + data.+=((3, 1L, "Hi World")) + data.+=((3, 1L, "Hi World")) + data.+=((3, 1L, "Hi World")) + data.+=((4, 1L, "Hi World")) + data.+=((4, 1L, "Test")) + + val t = env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", t) + + // "1,1,3", "2,1,2", "3,1,1", "4,1,2" + val distinct = "SELECT a, COUNT(DISTINCT b) AS distinct_b, COUNT(DISTINCT c) AS distinct_c " + + "FROM MyTable GROUP BY a" + val nestedDistinct = s"SELECT distinct_b, COUNT(DISTINCT distinct_c) " + + s"FROM ($distinct) GROUP BY distinct_b" + + val result = tEnv.sqlQuery(nestedDistinct).toRetractStream[Row] + result.addSink(new StreamITCase.RetractingSink).setParallelism(1) + + env.execute() + + val expected = List("1,3") + assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) + } + @Test def testUnboundedGroupByCollect(): Unit = { From 70623d3f7f40f382bca26b3ba8c9cf0d9ffb63eb Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 22 Nov 2018 22:11:32 +0100 Subject: [PATCH 133/359] [FLINK-10951][tests] Set yarn.nodemanager.vmem-check-enabled to false --- flink-jepsen/src/jepsen/flink/hadoop.clj | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-jepsen/src/jepsen/flink/hadoop.clj b/flink-jepsen/src/jepsen/flink/hadoop.clj index f633d07d621b7..99f3f72d45c3a 100644 --- a/flink-jepsen/src/jepsen/flink/hadoop.clj +++ b/flink-jepsen/src/jepsen/flink/hadoop.clj @@ -40,11 +40,14 @@ (defn yarn-site-config [test] - {:yarn.resourcemanager.hostname (resource-manager (:nodes test)) - :yarn.log-aggregation-enable "true" + {:yarn.log-aggregation-enable "true" + + :yarn.nodemanager.log-dirs yarn-log-dir :yarn.nodemanager.resource.cpu-vcores "8" + :yarn.nodemanager.vmem-check-enabled "false" + :yarn.resourcemanager.am.max-attempts "99999" - :yarn.nodemanager.log-dirs yarn-log-dir}) + :yarn.resourcemanager.hostname (resource-manager (:nodes test))}) (defn core-site-config [test] From 73b28a8540e9ba1d52669a7a4f8a2c4d7afbb428 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 9 Nov 2018 14:54:46 +0100 Subject: [PATCH 134/359] [FLINK-10842] [e2e] Fix broken waiting loops in common.sh This closes #7073. --- flink-end-to-end-tests/test-scripts/common.sh | 55 +++++++++++++------ .../test_queryable_state_restart_tm.sh | 1 + .../test_resume_externalized_checkpoints.sh | 3 +- 3 files changed, 40 insertions(+), 19 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 275d9c49f4b1e..645cf2a2f1e74 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -212,19 +212,22 @@ function start_local_zk { function wait_dispatcher_running { # wait at most 10 seconds until the dispatcher is up local QUERY_URL="${REST_PROTOCOL}://${NODENAME}:8081/taskmanagers" - for i in {1..10}; do + local TIMEOUT=10 + for i in $(seq 1 ${TIMEOUT}); do # without the || true this would exit our script if the JobManager is not yet up QUERY_RESULT=$(curl ${CURL_SSL_ARGS} "$QUERY_URL" 2> /dev/null || true) # ensure the taskmanagers field is there at all and is not empty if [[ ${QUERY_RESULT} =~ \{\"taskmanagers\":\[.+\]\} ]]; then echo "Dispatcher REST endpoint is up." - break + return fi echo "Waiting for dispatcher REST endpoint to come up..." sleep 1 done + echo "Dispatcher REST endpoint has not started within a timeout of ${TIMEOUT} sec" + exit 1 } function start_cluster { @@ -242,30 +245,45 @@ function start_taskmanagers { } function start_and_wait_for_tm { - local url="${REST_PROTOCOL}://${NODENAME}:8081/taskmanagers" - - tm_query_result=$(curl ${CURL_SSL_ARGS} -s "${url}") - + tm_query_result=`query_running_tms` # we assume that the cluster is running if ! [[ ${tm_query_result} =~ \{\"taskmanagers\":\[.*\]\} ]]; then echo "Your cluster seems to be unresponsive at the moment: ${tm_query_result}" 1>&2 exit 1 fi - running_tms=`curl ${CURL_SSL_ARGS} -s "${url}" | grep -o "id" | wc -l` - + running_tms=`query_number_of_running_tms` ${FLINK_DIR}/bin/taskmanager.sh start + wait_for_number_of_running_tms $((running_tms+1)) +} - for i in {1..10}; do - local new_running_tms=`curl ${CURL_SSL_ARGS} -s "${url}" | grep -o "id" | wc -l` - if [ $((new_running_tms-running_tms)) -eq 0 ]; then - echo "TaskManager is not yet up." +function query_running_tms { + local url="${REST_PROTOCOL}://${NODENAME}:8081/taskmanagers" + curl ${CURL_SSL_ARGS} -s "${url}" +} + +function query_number_of_running_tms { + query_running_tms | grep -o "id" | wc -l +} + +function wait_for_number_of_running_tms { + local TM_NUM_TO_WAIT=${1} + local TIMEOUT_COUNTER=10 + local TIMEOUT_INC=4 + local TIMEOUT=$(( $TIMEOUT_COUNTER * $TIMEOUT_INC )) + local TM_NUM_TEXT="Number of running task managers" + for i in $(seq 1 ${TIMEOUT_COUNTER}); do + local TM_NUM=`query_number_of_running_tms` + if [ $((TM_NUM - TM_NUM_TO_WAIT)) -eq 0 ]; then + echo "${TM_NUM_TEXT} has reached ${TM_NUM_TO_WAIT}." + return else - echo "TaskManager is up." - break + echo "${TM_NUM_TEXT} ${TM_NUM} is not yet ${TM_NUM_TO_WAIT}." fi - sleep 4 + sleep ${TIMEOUT_INC} done + echo "${TM_NUM_TEXT} has not reached ${TM_NUM_TO_WAIT} within a timeout of ${TIMEOUT} sec" + exit 1 } function check_logs_for_errors { @@ -376,17 +394,20 @@ function wait_for_job_state_transition { } function wait_job_running { - for i in {1..10}; do + local TIMEOUT=10 + for i in $(seq 1 ${TIMEOUT}); do JOB_LIST_RESULT=$("$FLINK_DIR"/bin/flink list -r | grep "$1") if [[ "$JOB_LIST_RESULT" == "" ]]; then echo "Job ($1) is not yet running." else echo "Job ($1) is running." - break + return fi sleep 1 done + echo "Job ($1) has not started within a timeout of ${TIMEOUT} sec" + exit 1 } function wait_job_terminal_state { diff --git a/flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh b/flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh index 56d811e14a1a9..db0174a2160cf 100755 --- a/flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh +++ b/flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh @@ -86,6 +86,7 @@ function run_test() { fi kill_random_taskmanager + wait_for_number_of_running_tms 0 latest_snapshot_count=$(cat $FLINK_DIR/log/*out* | grep "on snapshot" | tail -n 1 | awk '{print $4}') echo "Latest snapshot count was ${latest_snapshot_count}" diff --git a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh index 35fe30b6b25cf..48d68c98e9d68 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh @@ -100,11 +100,10 @@ fi DATASTREAM_JOB=$($JOB_CMD | grep "Job has been submitted with JobID" | sed 's/.* //g') -wait_job_running $DATASTREAM_JOB - if [[ $SIMULATE_FAILURE == "true" ]]; then wait_job_terminal_state $DATASTREAM_JOB FAILED else + wait_job_running $DATASTREAM_JOB wait_num_checkpoints $DATASTREAM_JOB 1 wait_oper_metric_num_in_records SemanticsCheckMapper.0 200 From 8cf611e525c9eb73177980ee94c5f2e67b72a8a5 Mon Sep 17 00:00:00 2001 From: azagrebin Date: Fri, 23 Nov 2018 17:36:16 +0100 Subject: [PATCH 135/359] [FLINK-10946] Silence checkpoint exception logging in task executor if job is not running --- .../flink/streaming/api/operators/AbstractStreamOperator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 04489b0ffd986..d9a195cb23553 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 @@ -416,7 +416,9 @@ public final OperatorSnapshotFutures snapshotState(long checkpointId, long times String snapshotFailMessage = "Could not complete snapshot " + checkpointId + " for operator " + getOperatorName() + "."; - LOG.info(snapshotFailMessage, snapshotException); + if (!getContainingTask().isCanceled()) { + LOG.info(snapshotFailMessage, snapshotException); + } throw new Exception(snapshotFailMessage, snapshotException); } From 461076b3e276049f14ee0a398dd634f7e4425e1a Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Wed, 21 Nov 2018 20:00:05 +0800 Subject: [PATCH 136/359] [FLINK-10009][table] Fix the casting problem for built-in TIMESTAMPADD. This closes #7155 --- .../table/codegen/calls/ScalarOperators.scala | 11 +++- .../expressions/ScalarFunctionsTest.scala | 63 +++++++++++++++++-- 2 files changed, 66 insertions(+), 8 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala index 282b167e91429..5a185da4922f5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala @@ -839,8 +839,15 @@ object ScalarOperators { generateArithmeticOperator(op, nullCheck, l, left, right, config) case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MILLIS) => - generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) { - (l, r) => s"$l $op ((int) ($r / ${MILLIS_PER_DAY}L))" + resultType match { + case SqlTimeTypeInfo.DATE => + generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) { + (l, r) => s"$l $op ((int) ($r / ${MILLIS_PER_DAY}L))" + } + case SqlTimeTypeInfo.TIMESTAMP => + generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIMESTAMP, left, right) { + (l, r) => s"$l * ${MILLIS_PER_DAY}L $op $r" + } } case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MONTHS) => diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala index 60cdda1f4e40c..31c9b6c3d0167 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala @@ -2702,17 +2702,68 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { "TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))", "null") - testAllApis( - "2016-06-15".toDate + 1.day, - "'2016-06-15'.toDate + 1.day", - "TIMESTAMPADD(DAY, 1, DATE '2016-06-15')", - "2016-06-16") - testAllApis( Null(Types.SQL_TIMESTAMP) + 3.months, "Null(SQL_TIMESTAMP) + 3.months", "TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))", "null") + + // TIMESTAMPADD with DATE returns a TIMESTAMP value for sub-day intervals. + testAllApis("2016-06-15".toDate + 1.month, + "'2016-06-15'.toDate + 1.month", + "timestampadd(MONTH, 1, date '2016-06-15')", + "2016-07-15") + + testAllApis("2016-06-15".toDate + 1.day, + "'2016-06-15'.toDate + 1.day", + "timestampadd(DAY, 1, date '2016-06-15')", + "2016-06-16") + + testAllApis("2016-06-15".toTimestamp - 1.hour, + "'2016-06-15'.toTimestamp - 1.hour", + "timestampadd(HOUR, -1, date '2016-06-15')", + "2016-06-14 23:00:00.0") + + testAllApis("2016-06-15".toTimestamp + 1.minute, + "'2016-06-15'.toTimestamp + 1.minute", + "timestampadd(MINUTE, 1, date '2016-06-15')", + "2016-06-15 00:01:00.0") + + testAllApis("2016-06-15".toTimestamp - 1.second, + "'2016-06-15'.toTimestamp - 1.second", + "timestampadd(SQL_TSI_SECOND, -1, date '2016-06-15')", + "2016-06-14 23:59:59.0") + + testAllApis("2016-06-15".toTimestamp + 1.second, + "'2016-06-15'.toTimestamp + 1.second", + "timestampadd(SECOND, 1, date '2016-06-15')", + "2016-06-15 00:00:01.0") + + testAllApis(Null(Types.SQL_TIMESTAMP) + 1.second, + "Null(SQL_TIMESTAMP) + 1.second", + "timestampadd(SECOND, 1, cast(null as date))", + "null") + + testAllApis(Null(Types.SQL_TIMESTAMP) + 1.day, + "Null(SQL_TIMESTAMP) + 1.day", + "timestampadd(DAY, 1, cast(null as date))", + "null") + + // Round to the last day of previous month + testAllApis("2016-05-31".toDate + 1.month, + "'2016-05-31'.toDate + 1.month", + "timestampadd(MONTH, 1, date '2016-05-31')", + "2016-06-30") + + testAllApis("2016-01-31".toDate + 5.month, + "'2016-01-31'.toDate + 5.month", + "timestampadd(MONTH, 5, date '2016-01-31')", + "2016-06-30") + + testAllApis("2016-03-31".toDate - 1.month, + "'2016-03-31'.toDate - 1.month", + "timestampadd(MONTH, -1, date '2016-03-31')", + "2016-02-29") } // ---------------------------------------------------------------------------------------------- From fb8fc04ceeb828cb9ddc05a1f2d67c00a197b3d7 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 15:30:50 +0100 Subject: [PATCH 137/359] [FLINK-10998][metrics][ganglia] Remove Ganglia reporter --- docs/monitoring/metrics.md | 28 ------ flink-dist/src/main/assemblies/opt.xml | 7 -- flink-metrics/flink-metrics-ganglia/pom.xml | 97 ------------------- .../metrics/ganglia/GangliaReporter.java | 84 ---------------- flink-metrics/pom.xml | 1 - 5 files changed, 217 deletions(-) delete mode 100644 flink-metrics/flink-metrics-ganglia/pom.xml delete mode 100644 flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/ganglia/GangliaReporter.java diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index e482df2e79579..bfa0576c98caf 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -622,34 +622,6 @@ An example for such a list would be `host=localhost,job_name=MyJob,task_name=MyT The domain thus identifies a metric class, while the key-property list identifies one (or multiple) instances of that metric. -### Ganglia (org.apache.flink.metrics.ganglia.GangliaReporter) - -In order to use this reporter you must copy `/opt/flink-metrics-ganglia-{{site.version}}.jar` into the `/lib` folder -of your Flink distribution. - -Parameters: - -- `host` - the gmond host address configured under `udp_recv_channel.bind` in `gmond.conf` -- `port` - the gmond port configured under `udp_recv_channel.port` in `gmond.conf` -- `tmax` - soft limit for how long an old metric should be retained -- `dmax` - hard limit for how long an old metric should be retained -- `ttl` - time-to-live for transmitted UDP packets -- `addressingMode` - UDP addressing mode to use (UNICAST/MULTICAST) - -Example configuration: - -{% highlight yaml %} - -metrics.reporter.gang.class: org.apache.flink.metrics.ganglia.GangliaReporter -metrics.reporter.gang.host: localhost -metrics.reporter.gang.port: 8649 -metrics.reporter.gang.tmax: 60 -metrics.reporter.gang.dmax: 0 -metrics.reporter.gang.ttl: 1 -metrics.reporter.gang.addressingMode: MULTICAST - -{% endhighlight %} - ### Graphite (org.apache.flink.metrics.graphite.GraphiteReporter) In order to use this reporter you must copy `/opt/flink-metrics-graphite-{{site.version}}.jar` into the `/lib` folder diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml index 8d153766bb690..ba459b1670eb1 100644 --- a/flink-dist/src/main/assemblies/opt.xml +++ b/flink-dist/src/main/assemblies/opt.xml @@ -91,13 +91,6 @@ 0644 - - ../flink-metrics/flink-metrics-ganglia/target/flink-metrics-ganglia-${project.version}-jar-with-dependencies.jar - opt/ - flink-metrics-ganglia-${project.version}.jar - 0644 - - ../flink-metrics/flink-metrics-graphite/target/flink-metrics-graphite-${project.version}-jar-with-dependencies.jar opt/ diff --git a/flink-metrics/flink-metrics-ganglia/pom.xml b/flink-metrics/flink-metrics-ganglia/pom.xml deleted file mode 100644 index 1cec50e6c1a26..0000000000000 --- a/flink-metrics/flink-metrics-ganglia/pom.xml +++ /dev/null @@ -1,97 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-metrics - 1.7-SNAPSHOT - .. - - - flink-metrics-ganglia - flink-metrics-ganglia - - - - org.apache.flink - flink-annotations - ${project.version} - provided - - - - org.apache.flink - flink-metrics-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/ganglia/GangliaReporter.java b/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/ganglia/GangliaReporter.java deleted file mode 100644 index 87199011fc563..0000000000000 --- a/flink-metrics/flink-metrics-ganglia/src/main/java/org/apache/flink/metrics/ganglia/GangliaReporter.java +++ /dev/null @@ -1,84 +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.metrics.ganglia; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.dropwizard.ScheduledDropwizardReporter; -import org.apache.flink.metrics.MetricConfig; - -import com.codahale.metrics.ScheduledReporter; -import info.ganglia.gmetric4j.gmetric.GMetric; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; - -/** - * This class acts as a factory for the {@link com.codahale.metrics.ganglia.GangliaReporter} and allows using it as a - * Flink reporter. - */ -@PublicEvolving -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(MetricConfig 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); - - log.info("Configured GangliaReporter with {host:{}, port:{}, dmax:{}, tmax:{}, ttl:{}, addressingMode:{}}", - host, port, dMax, tMax, ttl, addressingMode); - return builder.build(gMetric); - } catch (IOException e) { - throw new RuntimeException("Error while instantiating GangliaReporter.", e); - } - } -} diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml index f4031b1288c3f..403f5162ebe3b 100644 --- a/flink-metrics/pom.xml +++ b/flink-metrics/pom.xml @@ -36,7 +36,6 @@ under the License. flink-metrics-core flink-metrics-dropwizard - flink-metrics-ganglia flink-metrics-graphite flink-metrics-jmx flink-metrics-prometheus From b6900dda0f10b29161e6c0ba8236f393f7feb881 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 26 Nov 2018 15:58:38 +0100 Subject: [PATCH 138/359] [FLINK-11005] Define flink-sql-client uber-jar dependencies via artifactSet Instead of including every dependency and then limiting the set of included files via a filter condition of the maven-shade-pluging, this commit defines an artifact set of included dependencies. That way we will properly include all files belonging to the listed dependencies (e.g. also the NOTICE file). This closes #7176. --- flink-libraries/flink-sql-client/pom.xml | 27 +++++++----------------- 1 file changed, 8 insertions(+), 19 deletions(-) diff --git a/flink-libraries/flink-sql-client/pom.xml b/flink-libraries/flink-sql-client/pom.xml index c0299b8f31ebb..823c8366cb31c 100644 --- a/flink-libraries/flink-sql-client/pom.xml +++ b/flink-libraries/flink-sql-client/pom.xml @@ -144,30 +144,19 @@ under the License. + shade-flink package shade - - - *:* - - - org/apache/calcite/** - org/apache/flink/calcite/shaded/** - org/apache/flink/table/** - org/apache/flink/cep/** - org.codehaus.commons.compiler.properties - org/codehaus/janino/** - org/codehaus/commons/** - META-INF/services/org.apache.flink.table.factories.TableFactory - - org/jline/** - com/fasterxml/jackson/** - - - + + + org.apache.flink:flink-table_${scala.binary.version} + org.apache.flink:flink-cep_${scala.binary.version} + org.jline:* + + From f9e2c6aa5a7ea82a3d9df42807e98419ca698e8e Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Tue, 27 Nov 2018 15:15:17 +0100 Subject: [PATCH 139/359] [FLINK-10992][tests] Revise Hadoop configuration. Do not use /tmp directory to place log files or the HDFS data directory. Reconfigure dfs.replication to 1 because file availability is irrelevant in tests. Increase heap size of HDFS DataNodes and NameNode. Change find-files! function to not fail if directory does not exist. --- flink-jepsen/src/jepsen/flink/hadoop.clj | 57 ++++++++++++++---------- flink-jepsen/src/jepsen/flink/utils.clj | 14 ++++++ 2 files changed, 48 insertions(+), 23 deletions(-) diff --git a/flink-jepsen/src/jepsen/flink/hadoop.clj b/flink-jepsen/src/jepsen/flink/hadoop.clj index 99f3f72d45c3a..4ba78593b44fa 100644 --- a/flink-jepsen/src/jepsen/flink/hadoop.clj +++ b/flink-jepsen/src/jepsen/flink/hadoop.clj @@ -19,12 +19,15 @@ [clojure.tools.logging :refer :all] [jepsen [control :as c] - [db :as db]] - [jepsen.control.util :as cu])) + [db :as db] + [util :refer [meh]]] + [jepsen.control.util :as cu] + [jepsen.flink.utils :as fu])) (def install-dir "/opt/hadoop") (def hadoop-conf-dir (str install-dir "/etc/hadoop")) -(def yarn-log-dir "/tmp/logs/yarn") +(def log-dir (str install-dir "/logs")) +(def yarn-log-dir (str log-dir "/yarn")) (defn name-node [nodes] @@ -51,7 +54,12 @@ (defn core-site-config [test] - {:fs.defaultFS (str "hdfs://" (name-node (:nodes test)) ":9000")}) + {:hadoop.tmp.dir (str install-dir "/tmp") + :fs.defaultFS (str "hdfs://" (name-node (:nodes test)) ":9000")}) + +(defn hdfs-site-config + [_] + {:dfs.replication "1"}) (defn property-value [property value] @@ -66,8 +74,23 @@ (xml/element :configuration {} (map (fn [[k v]] (property-value k v)) (seq config))))] - (c/exec :echo config-xml :> config-file) - )) + (c/exec :echo config-xml :> config-file))) + +(defn- write-hadoop-env! + "Configures additional environment variables in hadoop-env.sh" + [] + (let [env-vars ["export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64" + "export HADOOP_NAMENODE_OPTS=\"-Xms2G -Xmx2G $HADOOP_NAMENODE_OPTS\"" + "export HADOOP_DATANODE_OPTS=\"-Xms2G -Xmx2G $HADOOP_DATANODE_OPTS\""]] + (doseq [env-var env-vars] + (c/exec :echo env-var :>> (str install-dir "/etc/hadoop/hadoop-env.sh"))))) + +(defn- write-configuration! + [test] + (write-config! (str install-dir "/etc/hadoop/yarn-site.xml") (yarn-site-config test)) + (write-config! (str install-dir "/etc/hadoop/core-site.xml") (core-site-config test)) + (write-config! (str install-dir "/etc/hadoop/hdfs-site.xml") (hdfs-site-config test)) + (write-hadoop-env!)) (defn start-name-node! [test node] @@ -104,12 +127,6 @@ (info "Start NodeManager") (c/exec (str install-dir "/sbin/yarn-daemon.sh") :--config hadoop-conf-dir :start :nodemanager))) -(defn find-files! - [dir] - (->> - (clojure.string/split (c/exec :find dir :-type :f) #"\n") - (remove clojure.string/blank?))) - (defn db [url] (reify db/DB @@ -117,26 +134,20 @@ (info "Install Hadoop from" url) (c/su (cu/install-archive! url install-dir) - (write-config! (str install-dir "/etc/hadoop/yarn-site.xml") (yarn-site-config test)) - (write-config! (str install-dir "/etc/hadoop/core-site.xml") (core-site-config test)) - (c/exec :echo (c/lit "export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64") :>> (str install-dir "/etc/hadoop/hadoop-env.sh")) + (write-configuration! test) (start-name-node-formatted! test node) (start-data-node! test node) (start-resource-manager! test node) (start-node-manager! test node))) - (teardown! [_ test node] + (teardown! [_ _ _] (info "Teardown Hadoop") (c/su (cu/grepkill! "hadoop") - (c/exec (c/lit (str "rm -rf /tmp/hadoop-* ||:"))))) + (c/exec :rm :-rf install-dir))) db/LogFiles (log-files [_ _ _] (c/su - (concat (find-files! (str install-dir "/logs")) - (if (cu/exists? yarn-log-dir) - (do - (c/exec :chmod :-R :777 yarn-log-dir) - (find-files! yarn-log-dir)) - [])))))) + (meh (c/exec :chmod :-R :755 log-dir)) + (fu/find-files! log-dir))))) diff --git a/flink-jepsen/src/jepsen/flink/utils.clj b/flink-jepsen/src/jepsen/flink/utils.clj index 2c7328f21a8d8..1aa53efe7aeab 100644 --- a/flink-jepsen/src/jepsen/flink/utils.clj +++ b/flink-jepsen/src/jepsen/flink/utils.clj @@ -51,6 +51,20 @@ (recur op (assoc keys :retries (dec retries)))) (success r))))) +(defn find-files! + "Lists files recursively given a directory. If the directory does not exist, an empty collection + is returned." + [dir] + (let [files (try + (c/exec :find dir :-type :f) + (catch Exception e + (if (.contains (.getMessage e) "No such file or directory") + "" + (throw e))))] + (->> + (clojure.string/split files #"\n") + (remove clojure.string/blank?)))) + ;;; runit process supervisor (http://smarden.org/runit/) (def runit-version "2.1.2-3") From 53f84f6992a30f02da547f382bb552008752c484 Mon Sep 17 00:00:00 2001 From: zentol Date: Sun, 25 Nov 2018 21:21:38 +0100 Subject: [PATCH 140/359] [FLINK-4173][metrics] Replace assembly-plugin usage --- flink-dist/src/main/assemblies/opt.xml | 9 +------- .../flink-metrics-dropwizard/pom.xml | 23 ------------------- flink-metrics/flink-metrics-graphite/pom.xml | 22 ++++++++++-------- 3 files changed, 14 insertions(+), 40 deletions(-) diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml index ba459b1670eb1..0d9acf3c3ae0e 100644 --- a/flink-dist/src/main/assemblies/opt.xml +++ b/flink-dist/src/main/assemblies/opt.xml @@ -85,14 +85,7 @@ - ../flink-metrics/flink-metrics-dropwizard/target/flink-metrics-dropwizard-${project.version}-jar-with-dependencies.jar - opt/ - flink-metrics-dropwizard-${project.version}.jar - 0644 - - - - ../flink-metrics/flink-metrics-graphite/target/flink-metrics-graphite-${project.version}-jar-with-dependencies.jar + ../flink-metrics/flink-metrics-graphite/target/flink-metrics-graphite-${project.version}.jar opt/ flink-metrics-graphite-${project.version}.jar 0644 diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml index 4ef94264346c4..b84aeed9e8b9c 100644 --- a/flink-metrics/flink-metrics-dropwizard/pom.xml +++ b/flink-metrics/flink-metrics-dropwizard/pom.xml @@ -84,27 +84,4 @@ under the License. - - - - - maven-assembly-plugin - 2.4 - - - jar-with-dependencies - - - - - make-assembly - package - - single - - - - - - diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml index eeb7f8b08446c..37cee5e1ddc55 100644 --- a/flink-metrics/flink-metrics-graphite/pom.xml +++ b/flink-metrics/flink-metrics-graphite/pom.xml @@ -69,20 +69,24 @@ under the License. - maven-assembly-plugin - 2.4 - - - jar-with-dependencies - - + org.apache.maven.plugins + maven-shade-plugin - make-assembly + shade-flink package - single + shade + + + + org.apache.flink:flink-metrics-dropwizard + io.dropwizard.metrics:metrics-core + io.dropwizard.metrics:metrics-graphite + + + From a6dd16aef16b3afd5fa0cd100709629dcf88e2a5 Mon Sep 17 00:00:00 2001 From: Aitozi Date: Wed, 28 Nov 2018 00:04:42 +0800 Subject: [PATCH 141/359] [FLINK-10990][metrics] Enforce minimum timespan in MeterView --- .../main/java/org/apache/flink/metrics/MeterView.java | 6 +++++- .../java/org/apache/flink/metrics/MeterViewTest.java | 11 +++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java index 8df0e868f8f8e..6d53325edafe3 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java @@ -50,7 +50,11 @@ public MeterView(int timeSpanInSeconds) { public MeterView(Counter counter, int timeSpanInSeconds) { this.counter = counter; - this.timeSpanInSeconds = timeSpanInSeconds - (timeSpanInSeconds % UPDATE_INTERVAL_SECONDS); + // the time-span must be larger than the update-interval as otherwise the array has a size of 1, + // for which no rate can be computed as no distinct before/after measurement exists. + this.timeSpanInSeconds = Math.max( + timeSpanInSeconds - (timeSpanInSeconds % UPDATE_INTERVAL_SECONDS), + UPDATE_INTERVAL_SECONDS); this.values = new long[this.timeSpanInSeconds / UPDATE_INTERVAL_SECONDS + 1]; } diff --git a/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java b/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java index a7a63b099493a..7752a6e80bb32 100644 --- a/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java +++ b/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java @@ -20,6 +20,7 @@ import org.junit.Test; +import static org.apache.flink.metrics.View.UPDATE_INTERVAL_SECONDS; import static org.junit.Assert.assertEquals; /** @@ -94,4 +95,14 @@ public void testGetRate() { assertEquals(0.0, m.getRate(), 0.1); // 480 - 480 / 60 } + + @Test + public void testTimeSpanBelowUpdateRate() { + int timeSpanInSeconds = 1; + MeterView m = new MeterView(timeSpanInSeconds); + assert timeSpanInSeconds < UPDATE_INTERVAL_SECONDS; + m.markEvent(); + m.update(); + assertEquals(0.2, m.getRate(), 0.0); + } } From 6493ab7805b4cbae93273fe7fdf42daa03950755 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 16 Nov 2018 22:35:14 +0800 Subject: [PATCH 142/359] [FLINK-9574] [doc] Rework docuementation for custom state serializers and state evolution This closes #7124. --- docs/dev/migration.md | 11 + docs/dev/stream/state/custom_serialization.md | 316 +++++++++++------- docs/dev/stream/state/index.md | 3 +- docs/dev/stream/state/schema_evolution.md | 93 ++++++ 4 files changed, 305 insertions(+), 118 deletions(-) create mode 100644 docs/dev/stream/state/schema_evolution.md diff --git a/docs/dev/migration.md b/docs/dev/migration.md index f91a3d8dea990..d472653cbb38d 100644 --- a/docs/dev/migration.md +++ b/docs/dev/migration.md @@ -25,6 +25,17 @@ under the License. * This will be replaced by the TOC {:toc} +## Migrating from Flink 1.3+ to Flink 1.7 + +### API changes for serializer snapshots + +This would be relevant mostly for users implementing custom `TypeSerializer`s for their state. + +The old `TypeSerializerConfigSnapshot` abstraction is now deprecated, and will be fully removed in the future +in favor of the new `TypeSerializerSnapshot`. For details and guides on how to migrate, please see +[Migrating from deprecated serializer snapshot APIs before Flink 1.7] +({{ site.baseurl }}/dev/stream/state/custom_serialization.html#migration-from-deprecated-serializer-snapshot-apis-before-Flink-1.7). + ## Migrating from Flink 1.2 to Flink 1.3 There are a few APIs that have been changed since Flink 1.2. Most of the changes are documented in their diff --git a/docs/dev/stream/state/custom_serialization.md b/docs/dev/stream/state/custom_serialization.md index f3941bc0c982a..cc727bef6c2d1 100644 --- a/docs/dev/stream/state/custom_serialization.md +++ b/docs/dev/stream/state/custom_serialization.md @@ -1,8 +1,8 @@ --- title: "Custom Serialization for Managed State" -nav-title: "Custom Serialization" +nav-title: "Custom State Serialization" nav-parent_id: streaming_state -nav-pos: 6 +nav-pos: 7 --- -If your application uses Flink's managed state, it might be necessary to implement custom serialization logic for special use cases. +* ToC +{:toc} -This page is targeted as a guideline for users who require the use of custom serialization for their state, covering how -to provide a custom serializer and how to handle upgrades to the serializer for compatibility. If you're simply using -Flink's own serializers, this page is irrelevant and can be skipped. +This page is targeted as a guideline for users who require the use of custom serialization for their state, covering +how to provide a custom state serializer as well as guidelines and best practices for implementing serializers that allow +state schema evolution. -### Using custom serializers +If you're simply using Flink's own serializers, this page is irrelevant and can be ignored. -As demonstrated in the above examples, when registering a managed operator or keyed state, a `StateDescriptor` is required +## Using custom state serializers + +When registering a managed operator or keyed state, a `StateDescriptor` is required to specify the state's name, as well as information about the type of the state. The type information is used by Flink's [type serialization framework](../../types_serialization.html) to create appropriate serializers for the state. @@ -66,125 +69,204 @@ checkpointedState = getRuntimeContext.getListState(descriptor) -Note that Flink writes state serializers along with the state as metadata. In certain cases on restore (see following -subsections), the written serializer needs to be deserialized and used. Therefore, it is recommended to avoid using -anonymous classes as your state serializers. Anonymous classes do not have a guarantee on the generated classname, -which varies across compilers and depends on the order that they are instantiated within the enclosing class, which can -easily cause the previously written serializer to be unreadable (since the original class can no longer be found in the -classpath). - -### Handling serializer upgrades and compatibility - -Flink allows changing the serializers used to read and write managed state, so that users are not locked in to any -specific serialization. When state is restored, the new serializer registered for the state (i.e., the serializer -that comes with the `StateDescriptor` used to access the state in the restored job) will be checked for compatibility, -and is replaced as the new serializer for the state. - -A compatible serializer would mean that the serializer is capable of reading previous serialized bytes of the state, -and the written binary format of the state also remains identical. The means to check the new serializer's compatibility -is provided through the following two methods of the `TypeSerializer` interface: - -{% highlight java %} -public abstract TypeSerializerConfigSnapshot snapshotConfiguration(); -public abstract CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); -{% endhighlight %} - -Briefly speaking, every time a checkpoint is performed, the `snapshotConfiguration` method is called to create a -point-in-time view of the state serializer's configuration. The returned configuration snapshot is stored along with the -checkpoint as the state's metadata. When the checkpoint is used to restore a job, that serializer configuration snapshot -will be provided to the _new_ serializer of the same state via the counterpart method, `ensureCompatibility`, to verify -compatibility of the new serializer. This method serves as a check for whether or not the new serializer is compatible, -as well as a hook to possibly reconfigure the new serializer in the case that it is incompatible. +## State serializers and schema evolution -Note that Flink's own serializers are implemented such that they are at least compatible with themselves, i.e. when the -same serializer is used for the state in the restored job, the serializer's will reconfigure themselves to be compatible -with their previous configuration. +This section explains the user-facing abstractions related to state serialization and schema evolution, and necessary +internal details about how Flink interacts with these abstractions. -The following subsections illustrate guidelines to implement these two methods when using custom serializers. +When restoring from savepoints, Flink allows changing the serializers used to read and write previously registered state, +so that users are not locked in to any specific serialization schema. When state is restored, a new serializer will be +registered for the state (i.e., the serializer that comes with the `StateDescriptor` used to access the state in the +restored job). This new serializer may have a different schema than that of the previous serializer. Therefore, when +implementing state serializers, besides the basic logic of reading / writing data, another important thing to keep in +mind is how the serialization schema can be changed in the future. -#### Implementing the `snapshotConfiguration` method +When speaking of *schema*, in this context the term is interchangeable between referring to the *data model* of a state +type and the *serialized binary format* of a state type. The schema, generally speaking, can change for a few cases: -The serializer's configuration snapshot should capture enough information such that on restore, the information -carried over to the new serializer for the state is sufficient for it to determine whether or not it is compatible. -This could typically contain information about the serializer's parameters or binary format of the serialized data; -generally, anything that allows the new serializer to decide whether or not it can be used to read previous serialized -bytes, and that it writes in the same binary format. + 1. Data schema of the state type has evolved, i.e. adding or removing a field from a POJO that is used as state. + 2. Generally speaking, after a change to the data schema, the serialization format of the serializer will need to be upgraded. + 3. Configuration of the serializer has changed. + +In order for the new execution to have information about the *written schema* of state and detect whether or not the +schema has changed, upon taking a savepoint of an operator's state, a *snapshot* of the state serializer needs to be +written along with the state bytes. This is abstracted a `TypeSerializerSnapshot`, explained in the next subsection. -How the serializer's configuration snapshot is written to and read from checkpoints is fully customizable. The below -is the base class for all serializer configuration snapshot implementations, the `TypeSerializerConfigSnapshot`. +### The `TypeSerializerSnapshot` abstraction +
    {% highlight java %} -public abstract TypeSerializerConfigSnapshot extends VersionedIOReadableWritable { - public abstract int getVersion(); - public void read(DataInputView in) {...} - public void write(DataOutputView out) {...} +public interface TypeSerializerSnapshot { + int getCurrentVersion(); + void writeSnapshot(DataOuputView out) throws IOException; + void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException; + TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer); + TypeSerializer restoreSerializer(); } {% endhighlight %} +
    -The `read` and `write` methods define how the configuration is read from and written to the checkpoint. The base -implementations contain logic to read and write the version of the configuration snapshot, so it should be extended and -not completely overridden. - -The version of the configuration snapshot is determined through the `getVersion` method. Versioning for the serializer -configuration snapshot is the means to maintain compatible configurations, as information included in the configuration -may change over time. By default, configuration snapshots are only compatible with the current version (as returned by -`getVersion`). To indicate that the configuration is compatible with other versions, override the `getCompatibleVersions` -method to return more version values. When reading from the checkpoint, you can use the `getReadVersion` method to -determine the version of the written configuration and adapt the read logic to the specific version. - -Attention The version of the serializer's configuration snapshot is **not** -related to upgrading the serializer. The exact same serializer can have different implementations of its -configuration snapshot, for example when more information is added to the configuration to allow more comprehensive -compatibility checks in the future. - -One limitation of implementing a `TypeSerializerConfigSnapshot` is that an empty constructor must be present. The empty -constructor is required when reading the configuration snapshot from checkpoints. - -#### Implementing the `ensureCompatibility` method - -The `ensureCompatibility` method should contain logic that performs checks against the information about the previous -serializer carried over via the provided `TypeSerializerConfigSnapshot`, basically doing one of the following: - - * Check whether the serializer is compatible, while possibly reconfiguring itself (if required) so that it may be - compatible. Afterwards, acknowledge with Flink that the serializer is compatible. - - * Acknowledge that the serializer is incompatible and that state migration is required before Flink can proceed with - using the new serializer. - -The above cases can be translated to code by returning one of the following from the `ensureCompatibility` method: - - * **`CompatibilityResult.compatible()`**: This acknowledges that the new serializer is compatible, or has been reconfigured to - be compatible, and Flink can proceed with the job with the serializer as is. - - * **`CompatibilityResult.requiresMigration()`**: This acknowledges that the serializer is incompatible, or cannot be - reconfigured to be compatible, and requires a state migration before the new serializer can be used. State migration - is performed by using the previous serializer to read the restored state bytes to objects, and then serialized again - using the new serializer. - - * **`CompatibilityResult.requiresMigration(TypeDeserializer deserializer)`**: This acknowledgement has equivalent semantics - to `CompatibilityResult.requiresMigration()`, but in the case that the previous serializer cannot be found or loaded - to read the restored state bytes for the migration, a provided `TypeDeserializer` can be used as a fallback resort. - -Attention Currently, as of Flink 1.3, if the result of the compatibility check -acknowledges that state migration needs to be performed, the job simply fails to restore from the checkpoint as state -migration is currently not available. The ability to migrate state will be introduced in future releases. - -### Managing `TypeSerializer` and `TypeSerializerConfigSnapshot` classes in user code - -Since `TypeSerializer`s and `TypeSerializerConfigSnapshot`s are written as part of checkpoints along with the state -values, the availability of the classes within the classpath may affect restore behaviour. - -`TypeSerializer`s are directly written into checkpoints using Java Object Serialization. In the case that the new -serializer acknowledges that it is incompatible and requires state migration, it will be required to be present to be -able to read the restored state bytes. Therefore, if the original serializer class no longer exists or has been modified -(resulting in a different `serialVersionUID`) as a result of a serializer upgrade for the state, the restore would -not be able to proceed. The alternative to this requirement is to provide a fallback `TypeDeserializer` when -acknowledging that state migration is required, using `CompatibilityResult.requiresMigration(TypeDeserializer deserializer)`. +
    +{% highlight java %} +public abstract class TypeSerializer { + + // ... + + public abstract TypeSerializerSnapshot snapshotConfiguration(); +} +{% endhighlight %} +
    -The class of `TypeSerializerConfigSnapshot`s in the restored checkpoint must exist in the classpath, as they are -fundamental components to compatibility checks on upgraded serializers and would not be able to be restored if the class -is not present. Since configuration snapshots are written to checkpoints using custom serialization, the implementation -of the class is free to be changed, as long as compatibility of the configuration change is handled using the versioning -mechanisms in `TypeSerializerConfigSnapshot`. +A serializer's `TypeSerializerSnapshot` is a point-in-time information that serves as the single source of truth about +the state serializer's write schema, as well as any additional information mandatory to restore a serializer that +would be identical to the given point-in-time. The logic about what should be written and read at restore time +as the serializer snapshot is defined in the `writeSnapshot` and `readSnapshot` methods. + +Note that the snapshot's own write schema may also need to change over time (e.g. when you wish to add more information +about the serializer to the snapshot). To facilitate this, snapshots are versioned, with the current version +number defined in the `getCurrentVersion` method. On restore, when the serializer snapshot is read from savepoints, +the version of the schema in which the snapshot was written in will be provided to the `readSnapshot` method so that +the read implementation can handle different versions. + +At restore time, the logic that detects whether or not the new serializer's schema has changed should be implemented in +the `resolveSchemaCompatibility` method. When previous registered state is registered again with new serializers in the +restored execution of an operator, the new serializer is provided to the previous serializer's snapshot via this method. +This method returns a `TypeSerializerSchemaCompatibility` representing the result of the compatibility resolution, +which can be one of the following: + + 1. **`TypeSerializerSchemaCompatibility.compatibleAsIs()`**: this result signals that the new serializer is compatible, + meaning that the new serializer has identical schema with the previous serializer. It is possible that the new + serializer has been reconfigured in the `resolveSchemaCompatibility` method so that it is compatible. + 2. **`TypeSerializerSchemaCompatibility.compatibleAfterMigration()`**: this result signals that the new serializer has a + different serialization schema, and it is possible to migrate from the old schema by using the previous serializer + (which recognizes the old schema) to read bytes into state objects, and then rewriting the object back to bytes with + the new serializer (which recognizes the new schema). + 3. **`TypeSerializerSchemaCompatibility.incompatible()`**: this result signals that the new serializer has a + different serialization schema, but it is not possible to migrate from the old schema. + +The last bit of detail is how the previous serializer is obtained in the case that migration is required. +Another important role of a serializer's `TypeSerializerSnapshot` is that it serves as a factory to restore +the previous serializer. More specifically, the `TypeSerializerSnapshot` should implement the `restoreSerializer` method +to instantiate a serializer instance that recognizes the previous serializer's schema and configuration, and can therefore +safely read data written by the previous serializer. + +### How Flink interacts with the `TypeSerializer` and `TypeSerializerSnapshot` abstractions + +To wrap up, this section concludes how Flink, or more specifically the state backends, interact with the +abstractions. The interaction is slightly different depending on the state backend, but this is orthogonal +to the implementation of state serializers and their serializer snapshots. + +#### Off-heap state backends (e.g. `RocksDBStateBackend`) + + 1. **Register new state with a state serializer that has schema _A_** + - the registered `TypeSerializer` for the state is used to read / write state on every state access. + - State is written in schema *A*. + 2. **Take a savepoint** + - The serializer snapshot is extracted via the `TypeSerializer#snapshotConfiguration` method. + - The serializer snapshot is written to the savepoint, as well as the already-serialized state bytes (with schema *A*). + 3. **Restored execution re-accesses restored state bytes with new state serializer that has schema _B_** + - The previous state serializer's snapshot is restored. + - State bytes are not deserialized on restore, only loaded back to the state backends (therefore, still in schema *A*). + - Upon receiving the new serializer, it is provided to the restored previous serializer's snapshot via the + `TypeSerializer#resolveSchemaCompatibility` to check for schema compatibility. + 4. **Migrate state bytes in backend from schema _A_ to schema _B_** + - If the compatibility resolution reflects that the schema has changed and migration is possible, schema migration is + performed. The previous state serializer which recognizes schema _A_ will be obtained from the serializer snapshot, via + `TypeSerializerSnapshot#restoreSerializer()`, and is used to deserialize state bytes to objects, which in turn + are re-written again with the new serializer, which recognizes schema _B_ to complete the migration. All entries + of the accessed state is migrated all-together before processing continues. + - If the resolution signals incompatibility, then the state access fails with an exception. + +#### Heap state backends (e.g. `MemoryStateBackend`, `FsStateBackend`) + + 1. **Register new state with a state serializer that has schema _A_** + - the registered `TypeSerializer` is maintained by the state backend. + 2. **Take a savepoint, serializing all state with schema _A_** + - The serializer snapshot is extracted via the `TypeSerializer#snapshotConfiguration` method. + - The serializer snapshot is written to the savepoint. + - State objects are now serialized to the savepoint, written in schema _A_. + 3. **On restore, deserialize state into objects in heap** + - The previous state serializer's snapshot is restored. + - The previous serializer, which recognizes schema _A_, is obtained from the serializer snapshot, via + `TypeSerializerSnapshot#restoreSerializer()`, and is used to deserialize state bytes to objects. + - From now on, all of the state is already deserialized. + 4. **Restored execution re-accesses previous state with new state serializer that has schema _B_** + - Upon receiving the new serializer, it is provided to the restored previous serializer's snapshot via the + `TypeSerializer#resolveSchemaCompatibility` to check for schema compatibility. + - If the compatibility check signals that migration is required, nothing happens in this case since for + heap backends, all state is already deserialized into objects. + - If the resolution signals incompatibility, then the state access fails with an exception. + 5. **Take another savepoint, serializing all state with schema _B_** + - Same as step 2., but now state bytes are all in schema _B_. + +## Implementation notes and best practices + +#### 1. Flink restores serializer snapshots by instantiating them with their classname + +A serializer's snapshot, being the single source of truth for how a registered state was serialized, serves as an +entry point to reading state in savepoints. In order to be able to restore and access previous state, the previous state +serializer's snapshot must be able to be restored. + +Flink restores serializer snapshots by first instantiating the `TypeSerializerSnapshot` with its classname (written +along with the snapshot bytes). Therefore, to avoid being subject to unintended classname changes or instantiation +failures, `TypeSerializerSnapshot` classes should: + + - avoid being implemented as anonymous classes or nested classes, + - have a public, nullary constructor for instantiation + +#### 2. Avoid sharing the same `TypeSerializerSnapshot` class across different serializers + +Since schema compatibility checks goes through the serializer snapshots, having multiple serializers returning +the same `TypeSerializerSnapshot` class as their snapshot would complicate the implementation for the +`TypeSerializerSnapshot#resolveSchemaCompatibility` and `TypeSerializerSnapshot#restoreSerializer()` method. + +This would also be a bad separation of concerns; a single serializer's serialization schema, +configuration, as well as how to restore it, should be consolidated in its own dedicated `TypeSerializerSnapshot` class. + +#### 3. Use the `CompositeSerializerSnapshot` utility for serializers that contain nested serializers + +There may be cases where a `TypeSerializer` relies on other nested `TypeSerializer`s; take for example Flink's +`TupleSerializer`, where it is configured with nested `TypeSerializer`s for the tuple fields. In this case, +the snapshot of the most outer serializer should also contain snapshots of the nested serializers. + +The `CompositeSerializerSnapshot` can be used specifically for this scenario. It wraps the logic of resolving +the overall schema compatibility check result for the composite serializer. +For an example of how it should be used, one can refer to Flink's +[ListSerializerSnapshot](https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerSnapshot.java) implementation. + +## Migrating from deprecated serializer snapshot APIs before Flink 1.7 + +This section is a guide for API migration from serializers and serializer snapshots that existed before Flink 1.7. + +Before Flink 1.7, serializer snapshots were implemented as a `TypeSerializerConfigSnapshot` (which is now deprecated, +and will eventually be removed in the future to be fully replaced by the new `TypeSerializerSnapshot` interface). +Moreover, the responsibility of serializer schema compatibility checks lived within the `TypeSerializer`, +implemented in the `TypeSerializer#ensureCompatibility(TypeSerializerConfigSnapshot)` method. + +Another major difference between the new and old abstractions is that the deprecated `TypeSerializerConfigSnapshot` +did not have the capability of instantiating the previous serializer. Therefore, in the case where your serializer +still returns a subclass of `TypeSerializerConfigSnapshot` as its snapshot, the serializer instance itself will always +be written to savepoints using Java serialization so that the previous serializer may be available at restore time. +This is very undesirable, since whether or not restoring the job will be successful is susceptible to availability +of the previous serializer's class, or in general, whether or not the serializer instance can be read back at restore +time using Java serialization. This means that you be limited to the same serializer for your state, +and could be problematic once you want to upgrade serializer classes or perform schema migration. + +To be future-proof and have flexibility to migrate your state serializers and schema, it is highly recommended to +migrate from the old abstractions. The steps to do this is as follows: + + 1. Implement a new subclass of `TypeSerializerSnapshot`. This will be the new snapshot for your serializer. + 2. Return the new `TypeSerializerSnapshot` as the serializer snapshot for your serializer in the + `TypeSerializer#snapshotConfiguration()` method. + 3. Restore the job from the savepoint that existed before Flink 1.7, and then take a savepoint again. + Note that at this step, the old `TypeSerializerConfigSnapshot` of the serializer must still exist in the classpath, + and the implementation for the `TypeSerializer#ensureCompatibility(TypeSerializerConfigSnapshot)` method must not be + removed. The purpose of this process is to replace the `TypeSerializerConfigSnapshot` written in old savepoints + with the newly implemented `TypeSerializerSnapshot` for the serializer. + 4. Once you have a savepoint taken with Flink 1.7, the savepoint will contain `TypeSerializerSnapshot` as the + state serializer snapshot, and the serializer instance will no longer be written in the savepoint. + At this point, it is now safe to remove all implementations of the old abstraction (remove the old + `TypeSerializerConfigSnapshot` implementation as will as the + `TypeSerializer#ensureCompatibility(TypeSerializerConfigSnapshot)` from the serializer). {% top %} diff --git a/docs/dev/stream/state/index.md b/docs/dev/stream/state/index.md index f9a24c3d0399d..10de348a76d25 100644 --- a/docs/dev/stream/state/index.md +++ b/docs/dev/stream/state/index.md @@ -52,6 +52,7 @@ Where to go next? * [The Broadcast State Pattern](broadcast_state.html): Explains how to connect a broadcast stream with a non-broadcast stream and use state to exchange information between them. * [Checkpointing](checkpointing.html): Describes how to enable and configure checkpointing for fault tolerance. * [Queryable State](queryable_state.html): Explains how to access state from outside of Flink during runtime. -* [Custom Serialization for Managed State](custom_serialization.html): Discusses custom serialization logic for state and its upgrades. +* [State Schema Evolution](schema_evolution.html): Shows how schema of state types can be evolved. +* [Custom Serialization for Managed State](custom_serialization.html): Discusses how to implement custom serializers, especially for schema evolution. {% top %} diff --git a/docs/dev/stream/state/schema_evolution.md b/docs/dev/stream/state/schema_evolution.md new file mode 100644 index 0000000000000..2fb10a74ff1f0 --- /dev/null +++ b/docs/dev/stream/state/schema_evolution.md @@ -0,0 +1,93 @@ +--- +title: "State Schema Evolution" +nav-parent_id: streaming_state +nav-pos: 6 +--- + + +* ToC +{:toc} + +Apache Flink streaming applications are typically designed to run indefinitely or for long periods of time. +As with all long-running services, the applications need to be updated to adapt to changing requirements. +This goes the same for data schemas that the applications work against; they evolve along with the application. + +This page provides an overview of how you can evolve your state type's data schema. +The current restrictions varies across different types and state structures (`ValueState`, `ListState`, etc.). + +Note that the information on this page is relevant only if you are using state serializers that are +generated by Flink's own [type serialization framework]({{ site.baseurl }}/dev/types_serialization.html). +That is, when declaring your state, the provided state descriptor is not configured to use a specific `TypeSerializer` +or `TypeInformation`, in which case Flink infers information about the state type: + +
    +{% highlight java %} +ListStateDescriptor descriptor = + new ListStateDescriptor<>( + "state-name", + MyPojoType.class); + +checkpointedState = getRuntimeContext().getListState(descriptor); +{% endhighlight %} +
    + +Under the hood, whether or not the schema of state can be evolved depends on the serializer used to read / write +persisted state bytes. Simply put, a registered state's schema can only be evolved if its serializer properly +supports it. This is handled transparently by serializers generated by Flink's type serialization framework +(current scope of support is listed [below]({{ site.baseurl }}/dev/stream/state/schema_evolution#supported-data-types-for-schema-evolution)). + +If you intend to implement a custom `TypeSerializer` for your state type and would like to learn how to implement +the serializer to support state schema evolution, please refer to +[Custom State Serialization]({{ site.baseurl }}/dev/stream/state/custom_serialization). +The documentation there also covers necessary internal details about the interplay between state serializers and Flink's +state backends to support state schema evolution. + +## Evolving state schema + +To evolve the schema of a given state type, you would take the following steps: + + 1. Take a savepoint of your Flink streaming job. + 2. Update state types in your application (e.g., modifying your Avro type schema). + 3. Restore the job from the savepoint. When accessing state for the first time, Flink will assess whether or not + the schema had been changed for the state, and migrate state schema if necessary. + +The process of migrating state to adapt to changed schemas happens automatically, and independently for each state. +This process is performed internally by Flink by first checking if the new serializer for the state has different +serialization schema than the previous serializer; if so, the previous serializer is used to read the state to objects, +and written back to bytes again with the new serializer. + +Further details about the migration process is out of the scope of this documentation; please refer to +[here]({{ site.baseurl }}/dev/stream/state/custom_serialization). + +## Supported data types for schema evolution + +Currently, schema evolution is supported only for Avro. Therefore, if you care about schema evolution for +state, it is currently recommended to always use Avro for state data types. + +There are plans to extend the support for more composite types, such as POJOs; for more details, +please refer to [FLINK-10897](https://issues.apache.org/jira/browse/FLINK-10897). + +### Avro types + +Flink fully supports evolving schema of Avro type state, as long as the schema change is considered compatible by +[Avro's rules for schema resolution](http://avro.apache.org/docs/current/spec.html#Schema+Resolution). + +One limitation is that Avro generated classes used as the state type cannot be relocated or have different +namespaces when the job is restored. From a7ec9cbf2fa20fece4f2ba926ba685dc43661e6a Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Wed, 28 Nov 2018 17:49:56 +0800 Subject: [PATCH 143/359] [FLINK-11003][docs] Fix type parameter in lambda docs --- docs/dev/java_lambdas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/java_lambdas.md b/docs/dev/java_lambdas.md index 4b306ac455d02..0b4a5cc069d7b 100644 --- a/docs/dev/java_lambdas.md +++ b/docs/dev/java_lambdas.md @@ -107,7 +107,7 @@ env.fromElements(1, 2, 3) .map(new MyTuple2Mapper()) .print(); -public static class MyTuple2Mapper extends MapFunction { +public static class MyTuple2Mapper extends MapFunction> { @Override public Tuple2 map(Integer i) { return Tuple2.of(i, i); From b272f20e94eb9b0bd5b0af92105826be2437d5b6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 20 Nov 2018 18:27:10 +0100 Subject: [PATCH 144/359] [FLINK-10955] Extend release notes for Apache Flink 1.7.0 --- docs/release-notes/flink-1.7.md | 100 ++++++++++++++++++++++++++++++++ 1 file changed, 100 insertions(+) diff --git a/docs/release-notes/flink-1.7.md b/docs/release-notes/flink-1.7.md index b26ef8695b772..ea2ae6f87e597 100644 --- a/docs/release-notes/flink-1.7.md +++ b/docs/release-notes/flink-1.7.md @@ -22,6 +22,97 @@ under the License. These release notes discuss important aspects, such as configuration, behavior, or dependencies, that changed between Flink 1.6 and Flink 1.7. Please read these notes carefully if you are planning to upgrade your Flink version to 1.7. +### Scala 2.12 support + +When using Scala `2.12` you might have to add explicit type annotations in places where they were not required when using Scala `2.11`. +This is an excerpt from the `TransitiveClosureNaive.scala` example in the Flink code base that shows the changes that could be required. + +Previous code: +``` +val terminate = prevPaths + .coGroup(nextPaths) + .where(0).equalTo(0) { + (prev, next, out: Collector[(Long, Long)]) => { + val prevPaths = prev.toSet + for (n <- next) + if (!prevPaths.contains(n)) out.collect(n) + } +} +``` + +With Scala `2.12` you have to change it to: +``` +val terminate = prevPaths + .coGroup(nextPaths) + .where(0).equalTo(0) { + (prev: Iterator[(Long, Long)], next: Iterator[(Long, Long)], out: Collector[(Long, Long)]) => { + val prevPaths = prev.toSet + for (n <- next) + if (!prevPaths.contains(n)) out.collect(n) + } +} +``` + +The reason for this is that Scala `2.12` changes how lambdas are implemented. +They now use the lambda support using SAM interfaces introduced in Java 8. +This makes some method calls ambiguous because now both Scala-style lambdas and SAMs are candidates for methods were it was previously clear which method would be invoked. + +### State evolution + +Before Flink 1.7, serializer snapshots were implemented as a `TypeSerializerConfigSnapshot` (which is now deprecated, and will eventually be removed in the future to be fully replaced by the new `TypeSerializerSnapshot` interface introduced in 1.7). +Moreover, the responsibility of serializer schema compatibility checks lived within the `TypeSerializer`, implemented in the `TypeSerializer#ensureCompatibility(TypeSerializerConfigSnapshot)` method. + +To be future-proof and to have flexibility to migrate your state serializers and schema, it is highly recommended to migrate from the old abstractions. +Details and migration guides can be found [here](https://ci.apache.org/projects/flink/flink-docs-master/dev/stream/state/custom_serialization.html). + +### Removal of the legacy mode + +Flink no longer supports the legacy mode. +If you depend on this, then please use Flink `1.6.x`. + +### Savepoints being used for recovery + +Savepoints are now used while recovering. +Previously when using exactly-once sink one could get into problems with duplicate output data when a failure occurred after a savepoint was taken but before the next checkpoint occurred. +This results in the fact that savepoints are no longer exclusively under the control of the user. +Savepoint should not be moved nor deleted if there was no newer checkpoint or savepoint taken. + +### MetricQueryService runs in separate thread pool + +The metric query service runs now in its own `ActorSystem`. +It needs consequently to open a new port for the query services to communicate with each other. +The [query service port]({{site.baseurl}}/ops/config.html#metrics-internal-query-service-port) can be configured in `flink-conf.yaml`. + +### Granularity of latency metrics + +The default granularity for latency metrics has been modified. +To restore the previous behavior users have to explicitly set the [granularity]({{site.baseurl}}/ops/config.html#metrics-latency-granularity) to `subtask`. + +### Latency marker activation + +Latency metrics are now disabled by default, which will affect all jobs that do not explicitly set the `latencyTrackingInterval` via `ExecutionConfig#setLatencyTrackingInterval`. +To restore the previous default behavior users have to configure the [latency interval]({{site.baseurl}}/ops/config.html#metrics-latency-interval) in `flink-conf.yaml`. + +### Relocation of Hadoop's Netty dependency + +We now also relocate Hadoop's Netty dependency from `io.netty` to `org.apache.flink.hadoop.shaded.io.netty`. +You can now bundle your own version of Netty into your job but may no longer assume that `io.netty` is present in the `flink-shaded-hadoop2-uber-*.jar` file. + +### Local recovery fixed + +With the improvements to Flink's scheduling, it can no longer happen that recoveries require more slots than before if local recovery is enabled. +Consequently, we encourage our users to enable [local recovery]({{site.baseurl}}/ops/config.html#state-backend-local-recovery) in `flink-conf.yaml`. + +### Support for multi slot TaskManagers + +Flink now properly supports `TaskManagers` with multiple slots. +Consequently, `TaskManagers` can now be started with an arbitrary number of slots and it is no longer recommended to start them with a single slot. + +### StandaloneJobClusterEntrypoint generates JobGraph with fixed JobID + +The `StandaloneJobClusterEntrypoint`, which is launched by the script `standalone-job.sh` and used for the job-mode container images, now starts all jobs with a fixed `JobID`. +Thus, in order to run a cluster in HA mode, one needs to set a different [cluster id]({{site.baseurl}}/ops/config.html#high-availability-cluster-id) for each job/cluster. + ### Scala shell does not work with Scala 2.12 @@ -39,4 +130,13 @@ In any other cases, it is highly recommended to remove the config option `jobman In order to avoid future problems, this feature has been removed from the documentation until it will be fixed. See [FLINK-10880](https://issues.apache.org/jira/browse/FLINK-10880) for more details. +### SQL over window preceding clause + +The over window `preceding` clause is now optional. +It defaults to `UNBOUNDED` if not specified. + +### OperatorSnapshotUtil writes v2 snapshots + +Snapshots created with `OperatorSnapshotUtil` are now written in the savepoint format `v2`. + {% top %} From 29a30b7f4b16c34fe9fa128134b1593f80cbbe3b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 11:54:01 +0100 Subject: [PATCH 145/359] [FLINK-10987] Deactive ApacheLicenseResourceTransformer to add vanilla LICENSE and NOTICE file to every module --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 7f176facc5890..8a9a411151d74 100644 --- a/pom.xml +++ b/pom.xml @@ -1514,9 +1514,9 @@ under the License. - - - + + Apache Flink + From 3d40a414657b8506b36dfc8bb127b8379d169ce7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 25 Nov 2018 13:02:26 +0100 Subject: [PATCH 146/359] [FLINK-10987] Exclude packaged LICENSEs from license check by the apache-rat-plugin --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 8a9a411151d74..79188656bdb33 100644 --- a/pom.xml +++ b/pom.xml @@ -1251,6 +1251,7 @@ under the License. **/packaged_licenses/LICENSE.*.txt + **/licenses/LICENSE* flink-runtime-web/web-dashboard/package.json From 3d528e13776e0219c161a50825878c5a38ba945c Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 16:32:23 +0100 Subject: [PATCH 147/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-streaming-python --- .../src/main/resources/META-INF/NOTICE | 10 ++++++++++ .../resources/META-INF/licenses/LICENSE.jython | 17 +++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE create mode 100644 flink-libraries/flink-streaming-python/src/main/resources/META-INF/licenses/LICENSE.jython diff --git a/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE b/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..f89bbcbf77d06 --- /dev/null +++ b/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE @@ -0,0 +1,10 @@ +flink-streaming-python +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Python Software Foundation License. (https://opensource.org/licenses/PythonSoftFoundation.php) +See bundled license files for details. + +- org.python:jython-standalone:2.7.0 diff --git a/flink-libraries/flink-streaming-python/src/main/resources/META-INF/licenses/LICENSE.jython b/flink-libraries/flink-streaming-python/src/main/resources/META-INF/licenses/LICENSE.jython new file mode 100644 index 0000000000000..1c2f7ffeb5bc4 --- /dev/null +++ b/flink-libraries/flink-streaming-python/src/main/resources/META-INF/licenses/LICENSE.jython @@ -0,0 +1,17 @@ +PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 + +1. This LICENSE AGREEMENT is between the Python Software Foundation ("PSF"), and the Individual or Organization ("Licensee") accessing and otherwise using this software ("Jython") in source or binary form and its associated documentation. + +2. Subject to the terms and conditions of this License Agreement, PSF hereby grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, analyze, test, perform and/or display publicly, prepare derivative works, distribute, and otherwise use Jython alone or in any derivative version, provided, however, that PSF's License Agreement and PSF's notice of copyright, i.e., "Copyright (c) 2007 Python Software Foundation; All Rights Reserved" are retained in Jython alone or in any derivative version prepared by Licensee. + +3. In the event Licensee prepares a derivative work that is based on or incorporates Jython or any part thereof, and wants to make the derivative work available to others as provided herein, then Licensee hereby agrees to include in any such work a brief summary of the changes made to Jython. + +4. PSF is making Jython available to Licensee on an "AS IS" basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF JYTHON WILL NOT INFRINGE ANY THIRD PARTY RIGHTS. + +5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF JYTHON FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING JYTHON, OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. + +6. This License Agreement will automatically terminate upon a material breach of its terms and conditions. + +7. Nothing in this License Agreement shall be deemed to create any relationship of agency, partnership, or joint venture between PSF and Licensee. This License Agreement does not grant permission to use PSF trademarks or trade name in a trademark sense to endorse or promote products or services of Licensee, or any third party. + +8. By copying, installing or otherwise using Jython, Licensee agrees to be bound by the terms and conditions of this License Agreement. From b956565e7f84c6f87349ba3a7b29db8c404fb391 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 17:43:49 +0100 Subject: [PATCH 148/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-sql-client --- .../src/main/resources/META-INF/NOTICE | 11 +++++++++++ .../main/resources/META-INF/licenses/LICENSE.jline | 7 +++++++ 2 files changed, 18 insertions(+) create mode 100644 flink-libraries/flink-sql-client/src/main/resources/META-INF/NOTICE create mode 100644 flink-libraries/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.jline diff --git a/flink-libraries/flink-sql-client/src/main/resources/META-INF/NOTICE b/flink-libraries/flink-sql-client/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..9b039bbe1b1f4 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/main/resources/META-INF/NOTICE @@ -0,0 +1,11 @@ +flink-sql-client +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- org.jline:jline-terminal:3.9.0 +- org.jline:jline-reader:3.9.0 diff --git a/flink-libraries/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.jline b/flink-libraries/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.jline new file mode 100644 index 0000000000000..e34763968c262 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.jline @@ -0,0 +1,7 @@ +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. From cc69a93d79dee87e6f4f1a2a5a540fc11fdf5582 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 17:44:06 +0100 Subject: [PATCH 149/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-table --- .../src/main/resources/META-INF/NOTICE | 75 ++++--------------- .../META-INF/licenses/LICENSE.janino | 31 ++++++++ 2 files changed, 47 insertions(+), 59 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/resources/META-INF/licenses/LICENSE.janino diff --git a/flink-libraries/flink-table/src/main/resources/META-INF/NOTICE b/flink-libraries/flink-table/src/main/resources/META-INF/NOTICE index bfe2737d6d110..54a253575ac5a 100644 --- a/flink-libraries/flink-table/src/main/resources/META-INF/NOTICE +++ b/flink-libraries/flink-table/src/main/resources/META-INF/NOTICE @@ -1,64 +1,21 @@ -=========================================================== - NOTICE corresponding to the section 4d of - The Apache License, Version 2.0, for - flink-table -=========================================================== +flink-table +Copyright 2014-2018 The Apache Software Foundation -The 'flink-table' module bundles certain dependencies into -the binary release artifacts. -Below are the license statements where required by the license. +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). -Janino version (org.codehaus.janino : janino) v. 3.0.7 -under the following license: - - Janino - An embedded Java[TM] compiler - - Copyright (c) 2001-2016, Arno Unkrig - Copyright (c) 2015-2016 TIBCO Software Inc. - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - 2. Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following - disclaimer in the documentation and/or other materials - provided with the distribution. - 3. Neither the name of JANINO nor the names of its contributors - may be used to endorse or promote products derived from this - software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER - IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR - OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN - IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) +- com.esri.geometry:esri-geometry-api:2.0.0 +- com.google.guava:guava:19.0 +- joda-time:joda-time:2.5 +- net.hydromatic:aggdesigner-algorithm:6.0 +- org.apache.calcite:calcite-core:1.17.0 +- org.apache.calcite:calcite-linq4j:1.17.0 +- org.apache.calcite.avatica:avatica-core:1.12.0 -Reflections (org.reflections : reflections) v. 0.9.10 -under the following license: +This project bundles the following dependencies under the BSD license. +See bundled license files for details - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - Version 2, December 2004 - - Copyright (C) 2004 Sam Hocevar - - Everyone is permitted to copy and distribute verbatim or modified - copies of this license document, and changing it is allowed as long - as the name is changed. - - DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE - TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION - - 0. You just DO WHAT THE FUCK YOU WANT TO. - - \ No newline at end of file +- org.codehaus.janino:janino:3.0.7 +- org.codehaus.janino:commons-compiler:3.0.7 diff --git a/flink-libraries/flink-table/src/main/resources/META-INF/licenses/LICENSE.janino b/flink-libraries/flink-table/src/main/resources/META-INF/licenses/LICENSE.janino new file mode 100644 index 0000000000000..ef871e2426218 --- /dev/null +++ b/flink-libraries/flink-table/src/main/resources/META-INF/licenses/LICENSE.janino @@ -0,0 +1,31 @@ +Janino - An embedded Java[TM] compiler + +Copyright (c) 2001-2016, Arno Unkrig +Copyright (c) 2015-2016 TIBCO Software Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials + provided with the distribution. + 3. Neither the name of JANINO nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER +IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR +OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN +IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. From 2eca8d19330255d60b80ff7b90911326f6097dd1 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 17:44:17 +0100 Subject: [PATCH 150/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-mesos --- .../src/main/resources/META-INF/NOTICE | 18 +++++++++++ .../META-INF/licenses/LICENSE.protobuf | 32 +++++++++++++++++++ 2 files changed, 50 insertions(+) create mode 100644 flink-mesos/src/main/resources/META-INF/NOTICE create mode 100644 flink-mesos/src/main/resources/META-INF/licenses/LICENSE.protobuf diff --git a/flink-mesos/src/main/resources/META-INF/NOTICE b/flink-mesos/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..145489a7a99d5 --- /dev/null +++ b/flink-mesos/src/main/resources/META-INF/NOTICE @@ -0,0 +1,18 @@ +flink-mesos +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.netflix.fenzo:fenzo-core:0.10.1 +- org.apache.mesos:mesos:1.0.1 +- com.fasterxml.jackson.core:jackson-annotations:2.4.0 +- com.fasterxml.jackson.core:jackson-core:2.4.5 +- com.fasterxml.jackson.core:jackson-databind:2.4.5 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.google.protobuf:protobuf-java:2.6.2 diff --git a/flink-mesos/src/main/resources/META-INF/licenses/LICENSE.protobuf b/flink-mesos/src/main/resources/META-INF/licenses/LICENSE.protobuf new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/flink-mesos/src/main/resources/META-INF/licenses/LICENSE.protobuf @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. From 4df6052d09fff7d042163eda01bce77b7bfeb10c Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 18:04:03 +0100 Subject: [PATCH 151/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-runtime-web --- flink-runtime-web/pom.xml | 3 + .../src/main/resources/META-INF/NOTICE | 40 ++++++++ .../META-INF/licenses/LICENSE.angular | 22 +++++ .../LICENSE.angular-drag-and-drop-list | 22 +++++ .../META-INF/licenses/LICENSE.angular-moment | 21 ++++ .../licenses/LICENSE.angular-ui-router | 21 ++++ .../META-INF/licenses/LICENSE.bootstrap | 21 ++++ .../resources/META-INF/licenses/LICENSE.d3 | 27 ++++++ .../resources/META-INF/licenses/LICENSE.dagre | 19 ++++ .../META-INF/licenses/LICENSE.dagre-d3 | 19 ++++ .../META-INF/licenses/LICENSE.ev-emitter | 7 ++ .../META-INF/licenses/LICENSE.font-awesome | 97 +++++++++++++++++++ .../META-INF/licenses/LICENSE.graphlib | 19 ++++ .../META-INF/licenses/LICENSE.imagesloaded | 7 ++ .../META-INF/licenses/LICENSE.jquery | 20 ++++ .../META-INF/licenses/LICENSE.lodash | 10 ++ .../META-INF/licenses/LICENSE.moment | 22 +++++ .../licenses/LICENSE.moment-duration-format | 21 ++++ .../resources/META-INF/licenses/LICENSE.qtip2 | 22 +++++ .../resources/META-INF/licenses/LICENSE.split | 32 ++++++ 20 files changed, 472 insertions(+) create mode 100644 flink-runtime-web/src/main/resources/META-INF/NOTICE create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-drag-and-drop-list create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-moment create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-ui-router create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.bootstrap create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.d3 create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre-d3 create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.ev-emitter create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.font-awesome create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.graphlib create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.imagesloaded create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.jquery create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.lodash create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment-duration-format create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.qtip2 create mode 100644 flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.split diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index f4c4d0a19655a..2f381649c87e9 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -128,6 +128,9 @@ under the License. web/** + + src/main/resources + diff --git a/flink-runtime-web/src/main/resources/META-INF/NOTICE b/flink-runtime-web/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..32506e6c888b4 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/NOTICE @@ -0,0 +1,40 @@ +flink-runtime-web +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- nvd3#1.8.4 + +This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) +See bundled license files for details. + +- angular:1.4.8 +- angular-drag-and-drop-list:1.4.0 +- angular-moment:0.10.3 +- angular-ui-router:0.2.15 +- bootstrap:3.3.6 +- dagre:0.7.5 +- dagre-d3:0.4.17 +- ev-emitter:1.1.1 +- font-awesome:4.5.0 (CSS) +- graphlib:1.0.7 +- imagesloaded:4.1.4 +- jquery:2.2.0 +- lodash:3.10.1 +- moment:2.10.6 +- moment-duration-format:1.3.0 +- qtip2:2.2.1 +- Split.js:1.0.6 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- d3:3.5.12 + +This project bundles the following dependencies under SIL OFL 1.1 license. (https://opensource.org/licenses/OFL-1.1) +See bundled license files for details. + +- font-awesome:4.5.0 (Font) diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular new file mode 100644 index 0000000000000..91f0644936811 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular @@ -0,0 +1,22 @@ +The MIT License + +Copyright (c) 2010-2018 Google, Inc. http://angularjs.org + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-drag-and-drop-list b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-drag-and-drop-list new file mode 100644 index 0000000000000..1787fc65befe1 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-drag-and-drop-list @@ -0,0 +1,22 @@ +The MIT License (MIT) + +Copyright (c) 2014 Marcel Juenemann +Copyright (c) 2014-2016 Google Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-moment b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-moment new file mode 100644 index 0000000000000..fe9db79b9113a --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-moment @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2013-2016 Uri Shaked and contributors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-ui-router b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-ui-router new file mode 100644 index 0000000000000..6413b092d70f7 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.angular-ui-router @@ -0,0 +1,21 @@ +The MIT License + +Copyright (c) 2013-2015 The AngularUI Team, Karsten Sperling + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.bootstrap b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.bootstrap new file mode 100644 index 0000000000000..6ca0ceecc5d79 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.bootstrap @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2011-2018 Twitter, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.d3 b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.d3 new file mode 100644 index 0000000000000..1d9d875edb469 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.d3 @@ -0,0 +1,27 @@ +Copyright 2010-2017 Mike Bostock +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the author nor the names of contributors may be used to + endorse or promote products derived from this software without specific prior + written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre new file mode 100644 index 0000000000000..e3c8f95557dbb --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre @@ -0,0 +1,19 @@ +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre-d3 b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre-d3 new file mode 100644 index 0000000000000..1d64ed68ce64d --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.dagre-d3 @@ -0,0 +1,19 @@ +Copyright (c) 2013 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.ev-emitter b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.ev-emitter new file mode 100644 index 0000000000000..3e30b1aeff26a --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.ev-emitter @@ -0,0 +1,7 @@ +Copyright © 2018 David DeSandro + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.font-awesome b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.font-awesome new file mode 100644 index 0000000000000..b9fb2c6e6ef06 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.font-awesome @@ -0,0 +1,97 @@ +The MIT license + +Copyright + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + + + SIL OPEN FONT LICENSE + +Version 1.1 - 26 February 2007 +PREAMBLE + +The goals of the Open Font License (OFL) are to stimulate worldwide +development of collaborative font projects, to support the font creation +efforts of academic and linguistic communities, and to provide a free and +open framework in which fonts may be shared and improved in partnership +with others. + +The OFL allows the licensed fonts to be used, studied, modified and +redistributed freely as long as they are not sold by themselves. The +fonts, including any derivative works, can be bundled, embedded, +redistributed and/or sold with any software provided that any reserved +names are not used by derivative works. The fonts and derivatives, +however, cannot be released under any other type of license. The +requirement for fonts to remain under this license does not apply +to any document created using the fonts or their derivatives. +DEFINITIONS + +"Font Software" refers to the set of files released by the Copyright +Holder(s) under this license and clearly marked as such. This may +include source files, build scripts and documentation. + +"Reserved Font Name" refers to any names specified as such after the +copyright statement(s). + +"Original Version" refers to the collection of Font Software components as +distributed by the Copyright Holder(s). + +"Modified Version" refers to any derivative made by adding to, deleting, +or substituting — in part or in whole — any of the components of the +Original Version, by changing formats or by porting the Font Software to a +new environment. + +"Author" refers to any designer, engineer, programmer, technical +writer or other person who contributed to the Font Software. +PERMISSION & CONDITIONS + +Permission is hereby granted, free of charge, to any person obtaining +a copy of the Font Software, to use, study, copy, merge, embed, modify, +redistribute, and sell modified and unmodified copies of the Font +Software, subject to the following conditions: + +1) Neither the Font Software nor any of its individual components, +in Original or Modified Versions, may be sold by itself. + +2) Original or Modified Versions of the Font Software may be bundled, +redistributed and/or sold with any software, provided that each copy +contains the above copyright notice and this license. These can be +included either as stand-alone text files, human-readable headers or +in the appropriate machine-readable metadata fields within text or +binary files as long as those fields can be easily viewed by the user. + +3) No Modified Version of the Font Software may use the Reserved Font +Name(s) unless explicit written permission is granted by the corresponding +Copyright Holder. This restriction only applies to the primary font name as +presented to the users. + +4) The name(s) of the Copyright Holder(s) or the Author(s) of the Font +Software shall not be used to promote, endorse or advertise any +Modified Version, except to acknowledge the contribution(s) of the +Copyright Holder(s) and the Author(s) or with their explicit written +permission. + +5) The Font Software, modified or unmodified, in part or in whole, +must be distributed entirely under this license, and must not be +distributed under any other license. The requirement for fonts to +remain under this license does not apply to any document created +using the Font Software. +TERMINATION + +This license becomes null and void if any of the above conditions are +not met. +DISCLAIMER + +THE FONT SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO ANY WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF COPYRIGHT, PATENT, TRADEMARK, OR OTHER RIGHT. IN NO EVENT SHALL THE +COPYRIGHT HOLDER BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +INCLUDING ANY GENERAL, SPECIAL, INDIRECT, INCIDENTAL, OR CONSEQUENTIAL +DAMAGES, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF THE USE OR INABILITY TO USE THE FONT SOFTWARE OR FROM +OTHER DEALINGS IN THE FONT SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.graphlib b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.graphlib new file mode 100644 index 0000000000000..e3c8f95557dbb --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.graphlib @@ -0,0 +1,19 @@ +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.imagesloaded b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.imagesloaded new file mode 100644 index 0000000000000..3e30b1aeff26a --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.imagesloaded @@ -0,0 +1,7 @@ +Copyright © 2018 David DeSandro + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.jquery b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.jquery new file mode 100644 index 0000000000000..e3dbacb999cef --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.jquery @@ -0,0 +1,20 @@ +Copyright JS Foundation and other contributors, https://js.foundation/ + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.lodash b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.lodash new file mode 100644 index 0000000000000..e3a30b4eafbab --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.lodash @@ -0,0 +1,10 @@ +The MIT License + +Copyright JS Foundation and other contributors + +Based on Underscore.js, copyright Jeremy Ashkenas, +DocumentCloud and Investigative Reporters & Editors + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/lodash/lodash \ No newline at end of file diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment new file mode 100644 index 0000000000000..8618b7333d6f5 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment @@ -0,0 +1,22 @@ +Copyright (c) JS Foundation and other contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment-duration-format b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment-duration-format new file mode 100644 index 0000000000000..06ec6fbe07368 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.moment-duration-format @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2017 vin-car + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.qtip2 b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.qtip2 new file mode 100644 index 0000000000000..0cfb790dc91d3 --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.qtip2 @@ -0,0 +1,22 @@ +Copyright (c) 2012 Craig Michael Thompson + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. diff --git a/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.split b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.split new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/flink-runtime-web/src/main/resources/META-INF/licenses/LICENSE.split @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. From 2a34e8f9a0f4d4b09e35233cf3e48fef7d8b0a42 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 18:26:06 +0100 Subject: [PATCH 152/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-runtime --- flink-runtime/pom.xml | 11 +++++ .../src/main/resources/META-INF/NOTICE | 47 +++++++++++++++++++ .../META-INF/licenses/LICENSE.base64 | 26 ++++++++++ .../META-INF/licenses/LICENSE.jsr166y | 26 ++++++++++ .../resources/META-INF/licenses/LICENSE.jzlib | 26 ++++++++++ .../META-INF/licenses/LICENSE.webbit | 38 +++++++++++++++ 6 files changed, 174 insertions(+) create mode 100644 flink-runtime/src/main/resources/META-INF/NOTICE create mode 100644 flink-runtime/src/main/resources/META-INF/licenses/LICENSE.base64 create mode 100644 flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jsr166y create mode 100644 flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jzlib create mode 100644 flink-runtime/src/main/resources/META-INF/licenses/LICENSE.webbit diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 3412fd4713d59..8ad6440b51ef2 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -567,6 +567,17 @@ under the License. + + io.netty:netty + + + META-INF/license + + META-INF/NOTICE.txt + + * diff --git a/flink-runtime/src/main/resources/META-INF/NOTICE b/flink-runtime/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..12cd1d4ccf5b4 --- /dev/null +++ b/flink-runtime/src/main/resources/META-INF/NOTICE @@ -0,0 +1,47 @@ +flink-runtime +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.typesafe.akka:akka-remote_2.11:2.4.20 +- io.netty:netty:3.10.6.Final +- org.apache.zookeeper:zookeeper:3.4.10 +- org.uncommons.maths:uncommons-maths:1.2.2a + +This project bundles io.netty:netty:3.10.6.Final from which it inherits the following notices: + +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * licenses/LICENSE.jsr166y (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.base64 (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.jzlib (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product contains a modified version of 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * licenses/LICENSE.webbit (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit diff --git a/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.base64 b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.base64 new file mode 100644 index 0000000000000..31ebc840539c1 --- /dev/null +++ b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.base64 @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuate of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jsr166y b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jsr166y new file mode 100644 index 0000000000000..b1c292b54cb2b --- /dev/null +++ b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jsr166y @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuity of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jzlib b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jzlib new file mode 100644 index 0000000000000..29ad562af0a5e --- /dev/null +++ b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.jzlib @@ -0,0 +1,26 @@ +Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.webbit b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.webbit new file mode 100644 index 0000000000000..ec5f348998bd4 --- /dev/null +++ b/flink-runtime/src/main/resources/META-INF/licenses/LICENSE.webbit @@ -0,0 +1,38 @@ +(BSD License: http://www.opensource.org/licenses/bsd-license) + +Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the +following conditions are met: + +* Redistributions of source code must retain the above + copyright notice, this list of conditions and the + following disclaimer. + +* Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the + following disclaimer in the documentation and/or other + materials provided with the distribution. + +* Neither the name of the Webbit nor the names of + its contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND +CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE +GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT +OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + From 2a854304cc5b5941c63c5d971d4245c070fd7883 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 18:20:01 +0100 Subject: [PATCH 153/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-test-utils --- .../flink-test-utils/pom.xml | 10 ++++- .../src/main/resources/META-INF/NOTICE | 40 +++++++++++++++++++ .../META-INF/licenses/LICENSE.base64 | 26 ++++++++++++ .../META-INF/licenses/LICENSE.jsr166y | 26 ++++++++++++ .../resources/META-INF/licenses/LICENSE.jzlib | 26 ++++++++++++ .../META-INF/licenses/LICENSE.webbit | 38 ++++++++++++++++++ pom.xml | 7 ++++ 7 files changed, 171 insertions(+), 2 deletions(-) create mode 100644 flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE create mode 100644 flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.base64 create mode 100644 flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jsr166y create mode 100644 flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jzlib create mode 100644 flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.webbit diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 6c32a436babc2..d1c829fe82d3e 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -143,11 +143,17 @@ under the License. org.apache.flink.shaded.testutils.org.jboss.netty - + - * + io.netty:netty META-INF/maven/io.netty/** + + META-INF/license + + META-INF/NOTICE.txt diff --git a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..07a9b6bb4283f --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE @@ -0,0 +1,40 @@ +flink-test-utils +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- io.netty:netty:3.10.6.Final + +This project bundles io.netty:netty:3.10.6.Final from which it inherits the following notices: + +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + * LICENSE: + * licenses/LICENSE.jsr166y (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + * LICENSE: + * licenses/LICENSE.base64 (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + * LICENSE: + * licenses/LICENSE.jzlib (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product contains a modified version of 'Webbit', a Java event based +WebSocket and HTTP server: + * LICENSE: + * licenses/LICENSE.webbit (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit diff --git a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.base64 b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.base64 new file mode 100644 index 0000000000000..31ebc840539c1 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.base64 @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuate of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jsr166y b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jsr166y new file mode 100644 index 0000000000000..b1c292b54cb2b --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jsr166y @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuity of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jzlib b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jzlib new file mode 100644 index 0000000000000..29ad562af0a5e --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.jzlib @@ -0,0 +1,26 @@ +Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.webbit b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.webbit new file mode 100644 index 0000000000000..ec5f348998bd4 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/licenses/LICENSE.webbit @@ -0,0 +1,38 @@ +(BSD License: http://www.opensource.org/licenses/bsd-license) + +Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the +following conditions are met: + +* Redistributions of source code must retain the above + copyright notice, this list of conditions and the + following disclaimer. + +* Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the + following disclaimer in the documentation and/or other + materials provided with the distribution. + +* Neither the name of the Webbit nor the names of + its contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND +CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE +GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT +OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + diff --git a/pom.xml b/pom.xml index 79188656bdb33..d0e6a77100a48 100644 --- a/pom.xml +++ b/pom.xml @@ -1499,6 +1499,13 @@ under the License. ** + + + io.netty:netty + + META-INF/LICENSE.txt + + From 4b1b124b9bc93503fc5e4449c2d90b1cc880c936 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 21:18:27 +0100 Subject: [PATCH 154/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-shaded-hadoop2 --- .../src/main/resources/META-INF/NOTICE | 31 ++++++++++++++++++ .../META-INF/licenses/LICENSE.protobuf | 32 +++++++++++++++++++ flink-shaded-hadoop/pom.xml | 12 +++++-- 3 files changed, 72 insertions(+), 3 deletions(-) create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/NOTICE create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/licenses/LICENSE.protobuf diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/NOTICE b/flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..15680c69694af --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/NOTICE @@ -0,0 +1,31 @@ +flink-shaded-hadoop2 +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.guava:guava:11.0.2 +- net.java.dev.jets3t:jets3t:0.9.0 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- commons-httpclient:commons-httpclient:3.1 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 + +The version of the actual Hadoop dependencies depend on the built. + +- org.apache.hadoop:hadoop-common +- org.apache.hadoop:hadoop-hdfs +- org.apache.hadoop:hadoop-mapreduce-client-core +- org.apache.hadoop:hadoop-yarn-client +- org.apache.hadoop:hadoop-yarn-common +- org.apache.hadoop:hadoop-annotations +- org.apache.hadoop:hadoop-auth +- org.apache.hadoop:hadoop-yarn-api + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.google.protobuf.protobuf-java:2.5.0 diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/licenses/LICENSE.protobuf b/flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/licenses/LICENSE.protobuf new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/src/main/resources/META-INF/licenses/LICENSE.protobuf @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml index 34b8fb9409729..b7027ae533658 100644 --- a/flink-shaded-hadoop/pom.xml +++ b/flink-shaded-hadoop/pom.xml @@ -105,14 +105,20 @@ under the License. META-INF/*.RSA + + commons-httpclient:commons-httpclient + + META-INF/LICENSE.txt + + - - - + + Apache Flink + From 435c834b3e68cef8a395e94071e0c2782599127b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 22:15:24 +0100 Subject: [PATCH 155/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-shaded-hadoop2-uber --- .../flink-shaded-hadoop2-uber/pom.xml | 6 +- .../src/main/resources/META-INF/NOTICE | 52 +++ .../META-INF/licenses/LICENSE.cddlv1.0 | 129 +++++++ .../META-INF/licenses/LICENSE.cddlv1.1 | 348 ++++++++++++++++++ .../resources/META-INF/licenses/LICENSE.jsch | 30 ++ .../META-INF/licenses/LICENSE.paranamer | 28 ++ .../META-INF/licenses/LICENSE.xmlenc | 27 ++ 7 files changed, 617 insertions(+), 3 deletions(-) create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.0 create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.1 create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.jsch create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.paranamer create mode 100644 flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.xmlenc diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/pom.xml index 4fa07e23f33e6..5ac239288d910 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/pom.xml @@ -113,9 +113,9 @@ under the License. - - - + + Apache Flink + diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..ad28a9bbcad64 --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE @@ -0,0 +1,52 @@ +flink-shaded-hadoop2-uber +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.jamesmurty.utils:java-xmlbuilder:0.4 +- commons-beanutils:commons-beanutils:1.9.3 +- commons-cli:commons-cli:1.3.1 +- commons-codec:commons-codec:1.10 +- commons-collections:commons-collections:3.2.2 +- commons-configuration:commons-configuration:1.7 +- commons-daemon:commons-daemon:1.0.13 +- commons-digester:commons-digester:1.8.1 +- commons-io:commons-io:2.4 +- commons-lang:commons-lang:2.6 +- commons-logging:commons-logging:1.1.3 +- commons-net:commons-net:3.1 +- org.apache.avro:avro:1.8.2 +- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-math3:3.5 +- org.apache.zookeeper:zookeeper:3.4.10 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 +- org.xerial.snappy:snappy-java:1.1.4 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.jcraft:jsch:0.1.42 +- com.thoughtworks.paranamer:paranamer:2.7 +- xmlenc:xmlenc:0.52 + +This project bundles the following dependencies under the Common Development and Distribution License (CDDL) 1.0. +See bundled license files for details. + +- javax.activation:activation:1.1 (http://java.sun.com/products/javabeans/jaf/index.jsp) +- javax.servlet:servlet-api:2.5 (https://repo1.maven.org/maven2/javax/servlet/servlet-api/2.5/) +- javax.xml.stream:stax-api:1.0-2 (http://central.maven.org/maven2/javax/xml/stream/stax-api/1.0-2/) + +This project bundles the following dependencies under the Common Development and Distribution License (CDDL) 1.1. +See bundled license files for details. + +- com.sun.jersey:jersey-client:1.9 (http://central.maven.org/maven2/com/sun/jersey/jersey-client/1.9/) +- javax.xml.bind:jaxb-api:2.2.2 (https://jaxb.dev.java.net/) + +This project bundles "org.tukaani:xz:1.5". +This Java implementation of XZ has been put into the public domain, thus you can do +whatever you want with it. All the files in the package have been written by Lasse Collin, +but some files are heavily based on public domain code written by Igor Pavlov. diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.0 b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.0 new file mode 100644 index 0000000000000..468e7516a87ff --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.0 @@ -0,0 +1,129 @@ +1. Definitions. + +1.1. "Contributor" means each individual or entity that creates or contributes to the creation of Modifications. + +1.2. "Contributor Version" means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + +1.3. "Covered Software" means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + +1.4. "Executable" means the Covered Software in any form other than Source Code. + +1.5. "Initial Developer" means the individual or entity that first makes Original Software available under this License. + +1.6. "Larger Work" means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + +1.7. "License" means this document. + +1.8. "Licensable" means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + +1.9. "Modifications" means the Source Code and Executable form of any of the following: + +A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + +B. Any new file that contains any part of the Original Software or previous Modification; or + +C. Any new file that is contributed or otherwise made available under the terms of this License. + +1.10. "Original Software" means the Source Code and Executable form of computer software code that is originally released under this License. + +1.11. "Patent Claims" means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + +1.12. "Source Code" means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + +1.13. "You" (or "Your") means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, "You" includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, "control" means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + +2.1. The Initial Developer Grant. + +Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + +(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + +(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + +(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + +(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. + +Conditioned upon Your compliance with Section 3.1 below and +subject to third party intellectual property claims, each +Contributor hereby grants You a world-wide, royalty-free, +non-exclusive license: + +(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + +(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. + +Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + +3.2. Modifications. + +The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + +3.3. Required Notices. + +You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + +3.4. Application of Additional Terms. + +You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + +3.5. Distribution of Executable Versions. + +You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + +3.6. Larger Works. + +You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. + +Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + +4.2. Effect of New Versions. + +You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + +4.3. Modified Versions. + +When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + +6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as "Participant") alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + +6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a "commercial item," as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer software" (as that term is defined at 48 C.F.R. ¤ 252.227-7014(a)(1)) and "commercial computer software documentation" as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.1 b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.1 new file mode 100644 index 0000000000000..75372e3d2c9a4 --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.cddlv1.1 @@ -0,0 +1,348 @@ +1. Definitions. + + 1.1. "Contributor" means each individual or entity that creates or + contributes to the creation of Modifications. + + 1.2. "Contributor Version" means the combination of the Original + Software, prior Modifications used by a Contributor (if any), and + the Modifications made by that particular Contributor. + + 1.3. "Covered Software" means (a) the Original Software, or (b) + Modifications, or (c) the combination of files containing Original + Software with files containing Modifications, in each case including + portions thereof. + + 1.4. "Executable" means the Covered Software in any form other than + Source Code. + + 1.5. "Initial Developer" means the individual or entity that first + makes Original Software available under this License. + + 1.6. "Larger Work" means a work which combines Covered Software or + portions thereof with code not governed by the terms of this License. + + 1.7. "License" means this document. + + 1.8. "Licensable" means having the right to grant, to the maximum + extent possible, whether at the time of the initial grant or + subsequently acquired, any and all of the rights conveyed herein. + + 1.9. "Modifications" means the Source Code and Executable form of + any of the following: + + A. Any file that results from an addition to, deletion from or + modification of the contents of a file containing Original Software + or previous Modifications; + + B. Any new file that contains any part of the Original Software or + previous Modification; or + + C. Any new file that is contributed or otherwise made available + under the terms of this License. + + 1.10. "Original Software" means the Source Code and Executable form + of computer software code that is originally released under this + License. + + 1.11. "Patent Claims" means any patent claim(s), now owned or + hereafter acquired, including without limitation, method, process, + and apparatus claims, in any patent Licensable by grantor. + + 1.12. "Source Code" means (a) the common form of computer software + code in which modifications are made and (b) associated + documentation included in or with such code. + + 1.13. "You" (or "Your") means an individual or a legal entity + exercising rights under, and complying with all of the terms of, + this License. For legal entities, "You" includes any entity which + controls, is controlled by, or is under common control with You. For + purposes of this definition, "control" means (a) the power, direct + or indirect, to cause the direction or management of such entity, + whether by contract or otherwise, or (b) ownership of more than + fifty percent (50%) of the outstanding shares or beneficial + ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, the Initial Developer + hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Initial Developer, to use, reproduce, + modify, display, perform, sublicense and distribute the Original + Software (or portions thereof), with or without Modifications, + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of + Original Software, to make, have made, use, practice, sell, and + offer for sale, and/or otherwise dispose of the Original Software + (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on + the date Initial Developer first distributes or otherwise makes the + Original Software available to a third party under the terms of this + License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is + granted: (1) for code that You delete from the Original Software, or + (2) for infringements caused by: (i) the modification of the + Original Software, or (ii) the combination of the Original Software + with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, each Contributor hereby + grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Contributor to use, reproduce, modify, + display, perform, sublicense and distribute the Modifications + created by such Contributor (or portions thereof), either on an + unmodified basis, with other Modifications, as Covered Software + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling + of Modifications made by that Contributor either alone and/or in + combination with its Contributor Version (or portions of such + combination), to make, use, sell, offer for sale, have made, and/or + otherwise dispose of: (1) Modifications made by that Contributor (or + portions thereof); and (2) the combination of Modifications made by + that Contributor with its Contributor Version (or portions of such + combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective + on the date Contributor first distributes or otherwise makes the + Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is + granted: (1) for any code that Contributor has deleted from the + Contributor Version; (2) for infringements caused by: (i) third + party modifications of Contributor Version, or (ii) the combination + of Modifications made by that Contributor with other software + (except as part of the Contributor Version) or other devices; or (3) + under Patent Claims infringed by Covered Software in the absence of + Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available + in Executable form must also be made available in Source Code form + and that Source Code form must be distributed only under the terms + of this License. You must include a copy of this License with every + copy of the Source Code form of the Covered Software You distribute + or otherwise make available. You must inform recipients of any such + Covered Software in Executable form as to how they can obtain such + Covered Software in Source Code form in a reasonable manner on or + through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are + governed by the terms of this License. You represent that You + believe Your Modifications are Your original creation(s) and/or You + have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that + identifies You as the Contributor of the Modification. You may not + remove or alter any copyright, patent or trademark notices contained + within the Covered Software, or any notices of licensing or any + descriptive text giving attribution to any Contributor or the + Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in + Source Code form that alters or restricts the applicable version of + this License or the recipients' rights hereunder. You may choose to + offer, and to charge a fee for, warranty, support, indemnity or + liability obligations to one or more recipients of Covered Software. + However, you may do so only on Your own behalf, and not on behalf of + the Initial Developer or any Contributor. You must make it + absolutely clear that any such warranty, support, indemnity or + liability obligation is offered by You alone, and You hereby agree + to indemnify the Initial Developer and every Contributor for any + liability incurred by the Initial Developer or such Contributor as a + result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under + the terms of this License or under the terms of a license of Your + choice, which may contain terms different from this License, + provided that You are in compliance with the terms of this License + and that the license for the Executable form does not attempt to + limit or alter the recipient's rights in the Source Code form from + the rights set forth in this License. If You distribute the Covered + Software in Executable form under a different license, You must make + it absolutely clear that any terms which differ from this License + are offered by You alone, not by the Initial Developer or + Contributor. You hereby agree to indemnify the Initial Developer and + every Contributor for any liability incurred by the Initial + Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with + other code not governed by the terms of this License and distribute + the Larger Work as a single product. In such a case, You must make + sure the requirements of this License are fulfilled for the Covered + Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or + new versions of this License from time to time. Each version will be + given a distinguishing version number. Except as provided in Section + 4.3, no one other than the license steward has the right to modify + this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the + Covered Software available under the terms of the version of the + License under which You originally received the Covered Software. If + the Initial Developer includes a notice in the Original Software + prohibiting it from being distributed or otherwise made available + under any subsequent version of the License, You must distribute and + make the Covered Software available under the terms of the version + of the License under which You originally received the Covered + Software. Otherwise, You may also choose to use, distribute or + otherwise make the Covered Software available under the terms of any + subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new + license for Your Original Software, You may create and use a + modified version of this License if You: (a) rename the license and + remove any references to the name of the license steward (except to + note that the license differs from this License); and (b) otherwise + make it clear that the license contains terms which differ from this + License. + +5. DISCLAIMER OF WARRANTY. + + COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, + WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, + INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE + IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR + NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF + THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE + DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY + OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, + REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN + ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS + AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate + automatically if You fail to comply with terms herein and fail to + cure such breach within 30 days of becoming aware of the breach. + Provisions which, by their nature, must remain in effect beyond the + termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding + declaratory judgment actions) against Initial Developer or a + Contributor (the Initial Developer or Contributor against whom You + assert such claim is referred to as "Participant") alleging that the + Participant Software (meaning the Contributor Version where the + Participant is a Contributor or the Original Software where the + Participant is the Initial Developer) directly or indirectly + infringes any patent, then any and all rights granted directly or + indirectly to You by such Participant, the Initial Developer (if the + Initial Developer is not the Participant) and all Contributors under + Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice + from Participant terminate prospectively and automatically at the + expiration of such 60 day notice period, unless if within such 60 + day period You withdraw Your claim with respect to the Participant + Software against such Participant either unilaterally or pursuant to + a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant + alleging that the Participant Software directly or indirectly + infringes any patent where such claim is resolved (such as by + license or settlement) prior to the initiation of patent + infringement litigation, then the reasonable value of the licenses + granted by such Participant under Sections 2.1 or 2.2 shall be taken + into account in determining the amount or value of any payment or + license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, + all end user licenses that have been validly granted by You or any + distributor hereunder prior to termination (excluding licenses + granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + + UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT + (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE + INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF + COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE + TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR + CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT + LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER + FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR + LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE + POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT + APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH + PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH + LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR + LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION + AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + + The Covered Software is a "commercial item," as that term is defined + in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer + software" (as that term is defined at 48 C.F.R. § + 252.227-7014(a)(1)) and "commercial computer software documentation" + as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent + with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 + (June 1995), all U.S. Government End Users acquire Covered Software + with only those rights set forth herein. This U.S. Government Rights + clause is in lieu of, and supersedes, any other FAR, DFAR, or other + clause or provision that addresses Government rights in computer + software under this License. + +9. MISCELLANEOUS. + + This License represents the complete agreement concerning subject + matter hereof. If any provision of this License is held to be + unenforceable, such provision shall be reformed only to the extent + necessary to make it enforceable. This License shall be governed by + the law of the jurisdiction specified in a notice contained within + the Original Software (except to the extent applicable law, if any, + provides otherwise), excluding such jurisdiction's conflict-of-law + provisions. Any litigation relating to this License shall be subject + to the jurisdiction of the courts located in the jurisdiction and + venue specified in a notice contained within the Original Software, + with the losing party responsible for costs, including, without + limitation, court costs and reasonable attorneys' fees and expenses. + The application of the United Nations Convention on Contracts for + the International Sale of Goods is expressly excluded. Any law or + regulation which provides that the language of a contract shall be + construed against the drafter shall not apply to this License. You + agree that You alone are responsible for compliance with the United + States export administration regulations (and the export control + laws and regulation of any other countries) when You use, distribute + or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + + As between Initial Developer and the Contributors, each party is + responsible for claims and damages arising, directly or indirectly, + out of its utilization of rights under this License and You agree to + work with Initial Developer and Contributors to distribute such + responsibility on an equitable basis. Nothing herein is intended or + shall be deemed to constitute any admission of liability. diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.jsch b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.jsch new file mode 100644 index 0000000000000..edd491dfbfbf5 --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.jsch @@ -0,0 +1,30 @@ +JSch 0.0.* was released under the GNU LGPL license. Later, we have switched +over to a BSD-style license. + +------------------------------------------------------------------------------ +Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.paranamer b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.paranamer new file mode 100644 index 0000000000000..522176fbca03e --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.paranamer @@ -0,0 +1,28 @@ +[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ] + + Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.xmlenc b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.xmlenc new file mode 100644 index 0000000000000..3a70c9bfcdadd --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/licenses/LICENSE.xmlenc @@ -0,0 +1,27 @@ +Copyright 2003-2005, Ernst de Haan +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. From 6c8f4d3995862ce3b2098213c7cf3e8635e5d6ac Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 22:47:26 +0100 Subject: [PATCH 156/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-shaded-yarn-tests --- .../src/main/resources/META-INF/NOTICE | 47 +++++++++++++++++++ .../resources/META-INF/licenses/LICENSE.asm | 27 +++++++++++ .../META-INF/licenses/LICENSE.protobuf | 32 +++++++++++++ 3 files changed, 106 insertions(+) create mode 100644 flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/NOTICE create mode 100644 flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.asm create mode 100644 flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.protobuf diff --git a/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/NOTICE b/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..88c4e4c67a5de --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/NOTICE @@ -0,0 +1,47 @@ +flink-shaded-yarn-tests +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.guava:guava:11.0.2 +- io.netty:netty:3.6.2.Final +- net.java.dev.jets3t:jets3t:0.9.0 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- commons-httpclient:commons-httpclient:3.1 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-jaxrs:1.8.3 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 +- org.codehaus.jackson:jackson-xc:1.8.3 + +The version of the actual Hadoop dependencies depend on the built. + +- org.apache.hadoop:hadoop-annotations +- org.apache.hadoop:hadoop-auth +- org.apache.hadoop:hadoop-common +- org.apache.hadoop:hadoop-hdfs +- org.apache.hadoop:hadoop-mapreduce-client-app +- org.apache.hadoop:hadoop-mapreduce-client-common +- org.apache.hadoop:hadoop-mapreduce-client-core +- org.apache.hadoop:hadoop-mapreduce-client-hs +- org.apache.hadoop:hadoop-mapreduce-client-jobclient +- org.apache.hadoop:hadoop-mapreduce-client-shuffle +- org.apache.hadoop:hadoop-minicluster +- org.apache.hadoop:hadoop-yarn-api +- org.apache.hadoop:hadoop-yarn-client +- org.apache.hadoop:hadoop-yarn-common +- org.apache.hadoop:hadoop-yarn-server-applicationhistoryservice +- org.apache.hadoop:hadoop-yarn-server-common +- org.apache.hadoop:hadoop-yarn-server-nodemanager +- org.apache.hadoop:hadoop-yarn-server-resourcemanager +- org.apache.hadoop:hadoop-yarn-server-tests +- org.apache.hadoop:hadoop-yarn-server-web-proxy + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- asm:asm:3.1 +- com.google.protobuf.protobuf-java:2.5.0 diff --git a/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.asm b/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.asm new file mode 100644 index 0000000000000..c71bb7bac5d4d --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.asm @@ -0,0 +1,27 @@ +ASM: a very small and fast Java bytecode manipulation framework +Copyright (c) 2000-2011 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. +3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.protobuf b/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.protobuf new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/flink-shaded-hadoop/flink-shaded-yarn-tests/src/main/resources/META-INF/licenses/LICENSE.protobuf @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. From 15a6443cf15f36eb5f6622674cdaefee9e7e53a2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 22:57:00 +0100 Subject: [PATCH 157/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-shaded-curator --- .../src/main/resources/META-INF/NOTICE | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 flink-shaded-curator/src/main/resources/META-INF/NOTICE diff --git a/flink-shaded-curator/src/main/resources/META-INF/NOTICE b/flink-shaded-curator/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..e6b455563d080 --- /dev/null +++ b/flink-shaded-curator/src/main/resources/META-INF/NOTICE @@ -0,0 +1,12 @@ +flink-shaded-curator +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.guava:guava:16.0.1 +- org.apache.curator:curator-client:2.12.0 +- org.apache.curator:curator-framework:2.12.0 +- org.apache.curator:curator-recipes:2.12.0 From 4b9c1810f3c6dae3d5a0e2a3d8a288f2ef708f3c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Nov 2018 23:39:31 +0100 Subject: [PATCH 158/359] [FLINK-10987] Do not deploy flink-yarn-tests module to repository --- flink-yarn-tests/pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index c09b0dd64ca53..7f02ef6f5b9fa 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -185,6 +185,15 @@ under the License. + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + org.apache.maven.plugins maven-shade-plugin From 78dddcca08d932c8b5134e47a6019c493a0ca375 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 25 Nov 2018 14:33:50 +0100 Subject: [PATCH 159/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-connector-twitter --- .../src/main/resources/META-INF/NOTICE | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE diff --git a/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..c4e49bc2270c3 --- /dev/null +++ b/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE @@ -0,0 +1,13 @@ +flink-connector-twitter +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.guava:guava:14.0.1 +- com.twitter:hbc-core:2.2.0 +- com.twitter:joauth:6.0.2 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 From eaae01b39ab981fc2658f4d0698260741dfd9a57 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Nov 2018 15:37:13 +0100 Subject: [PATCH 160/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-metrics-datadog --- .../src/main/resources/META-INF/NOTICE | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE diff --git a/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..e577eda4c2294 --- /dev/null +++ b/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE @@ -0,0 +1,10 @@ +flink-metrics-datadog +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.squareup.okhttp3:okhttp:3.7.0 +- com.squareup.okio:okio:1.12.0 From c6601b400633e2539338eace515c0b7e4b6c1694 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Nov 2018 15:41:05 +0100 Subject: [PATCH 161/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-metrics-graphite --- .../src/main/resources/META-INF/NOTICE | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE diff --git a/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..ef94ebed7582e --- /dev/null +++ b/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE @@ -0,0 +1,10 @@ +flink-metrics-graphite +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- io.dropwizard.metrics:metrics-core:3.1.5 +- io.dropwizard.metrics:metrics-graphite:3.1.5 From d36d15447c42bc4d9367f2ec03311eb957131981 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Nov 2018 15:44:17 +0100 Subject: [PATCH 162/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-metrics-prometheus --- .../src/main/resources/META-INF/NOTICE | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE diff --git a/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..dedb042af855e --- /dev/null +++ b/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE @@ -0,0 +1,12 @@ +flink-metrics-prometheus +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- io.prometheus:simpleclient:0.3.0 +- io.prometheus:simpleclient_common:0.3.0 +- io.prometheus:simpleclient_httpserver:0.3.0 +- io.prometheus:simpleclient_pushgateway:0.3.0 From 46ff7008e46c12397b0c5c7504e2440caf9ec2d2 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 23 Nov 2018 16:09:21 +0100 Subject: [PATCH 163/359] [FLINK-10987] Add license notices for flink-filesystems --- .../flink-fs-hadoop-shaded/pom.xml | 1 + .../src/main/resources/META-INF/NOTICE | 2577 +++++++++++++++++ .../resources/META-INF/licenses/LICENSE-re2j | 32 + .../META-INF/licenses/LICENSE-stax2api | 22 + flink-filesystems/flink-s3-fs-base/pom.xml | 1 + .../src/main/resources/META-INF/NOTICE | 24 + flink-filesystems/flink-s3-fs-presto/pom.xml | 1 + .../src/main/resources/META-INF/NOTICE | 26 + .../META-INF/licenses/LICENSE-hdrhistogram | 125 + 9 files changed, 2809 insertions(+) create mode 100644 flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE create mode 100644 flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-re2j create mode 100644 flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-stax2api create mode 100644 flink-filesystems/flink-s3-fs-base/src/main/resources/META-INF/NOTICE create mode 100644 flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE create mode 100644 flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/licenses/LICENSE-hdrhistogram diff --git a/flink-filesystems/flink-fs-hadoop-shaded/pom.xml b/flink-filesystems/flink-fs-hadoop-shaded/pom.xml index b8029a187f21a..e5411f1c32451 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/pom.xml +++ b/flink-filesystems/flink-fs-hadoop-shaded/pom.xml @@ -250,6 +250,7 @@ under the License. PropertyList-1.0.dtd META-INF/maven/** META-INF/services/javax.xml.stream.* + META-INF/LICENSE.txt diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..e4774491b4df6 --- /dev/null +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE @@ -0,0 +1,2577 @@ +flink-fs-hadoop-shaded +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hadoop:hadoop-annotations:3.1.0 +- org.apache.hadoop:hadoop-auth:3.1.0 +- org.apache.hadoop:hadoop-common:3.1.0 +- org.apache.htrace:htrace-core4:4.1.0-incubating +- org.apache.commons:commons-configuration2:2.1.1 +- org.apache.commons:commons-lang3:3.3.2 +- commons-lang:commons-lang:2.6 +- commons-collections:commons-collections:3.2.2 +- commons-io:commons-io:2.4 +- commons-logging:commons-logging:1.1.3 +- commons-beanutils:commons-beanutils:1.9.3 +- com.google.guava:guava:11.0.2 +- com.fasterxml.jackson.core:jackson-annotations:2.7.0 +- com.fasterxml.jackson.core:jackson-core:2.7.8 +- com.fasterxml.jackson.core:jackson-databind:2.7.8 +- com.fasterxml.woodstox:woodstox-core:5.0.3 + +This project bundles the following dependencies under the Go License (https://golang.org/LICENSE). +See bundled license files for details. + +- com.google.re2j:re2j:1.1 + +This project bundles the following dependencies under BSD License (https://opensource.org/licenses/bsd-license.php). +See bundled license files for details. + +- org.codehaus.woodstox:stax2-api:3.1.4 (https://github.com/FasterXML/stax2-api/tree/stax2-api-3.1.4) + +This project bundles org.apache.hadoop:*:3.1.0 from which it inherits the following notices: + +The Apache Hadoop project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + +For the org.apache.hadoop.util.bloom.* classes: + +/** + * + * Copyright (c) 2005, European Commission project OneLab under contract + * 034819 (http://www.one-lab.org) + * All rights reserved. + * Redistribution and use in source and binary forms, with or + * without modification, are permitted provided that the following + * conditions are met: + * - Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * - Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the distribution. + * - Neither the name of the University Catholique de Louvain - UCL + * nor the names of its contributors may be used to endorse or + * promote products derived from this software without specific prior + * written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN + * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +For portions of the native implementation of slicing-by-8 CRC calculation +in src/main/native/src/org/apache/hadoop/util: + +Copyright (c) 2008,2009,2010 Massachusetts Institute of Technology. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. +* Neither the name of the Massachusetts Institute of Technology nor + the names of its contributors may be used to endorse or promote + products derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +Other portions are under the same license from Intel: +http://sourceforge.net/projects/slicing-by-8/ +/*++ + * + * Copyright (c) 2004-2006 Intel Corporation - All Rights Reserved + * + * This software program is licensed subject to the BSD License, + * available at http://www.opensource.org/licenses/bsd-license.html + * + * Abstract: The main routine + * + --*/ + +For src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,lz4hc.c}, + +/* + LZ4 - Fast LZ compression algorithm + Header File + Copyright (C) 2011-2014, Yann Collet. + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - LZ4 source repository : http://code.google.com/p/lz4/ + - LZ4 public forum : https://groups.google.com/forum/#!forum/lz4c +*/ + + +For hadoop-common-project/hadoop-common/src/main/native/gtest +--------------------------------------------------------------------- +Copyright 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles these dependencies under the +following license: +re2j 1.1 +--------------------------------------------------------------------- +(GO license) +This is a work derived from Russ Cox's RE2 in Go, whose license +http://golang.org/LICENSE is as follows: + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/util/tree.h +--------------------------------------------------------------------- +Copyright 2002 Niels Provos +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles binaries of leveldbjni +(https://github.com/fusesource/leveldbjni), which is available under the +following license: + +Copyright (c) 2011 FuseSource Corp. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of FuseSource Corp. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/compat/{fstatat|openat|unlinkat}.h: + +Copyright (c) 2012 The FreeBSD Foundation +All rights reserved. + +This software was developed by Pawel Jakub Dawidek under sponsorship from +the FreeBSD Foundation. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHORS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHORS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS +OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + +============= + +The binary distribution of this product bundles binaries of leveldb +(http://code.google.com/p/leveldb/), which is available under the following +license: + +Copyright (c) 2011 The LevelDB Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles binaries of snappy +(http://code.google.com/p/snappy/), which is available under the following +license: + +Copyright 2011, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/ +-------------------------------------------------------------------------------- +Copyright (C) 2008-2016, SpryMedia Ltd. + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2010 Aleksander Williams + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2011-2016 Tim Wood, Iskren Chernev, Moment.js contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. + +The binary distribution of this product bundles these dependencies under the +following license: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2 +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css +bootstrap v3.3.6 +broccoli-asset-rev v2.4.2 +broccoli-funnel v1.0.1 +datatables v1.10.8 +em-helpers v0.5.13 +em-table v0.1.6 +ember v2.2.0 +ember-array-contains-helper v1.0.2 +ember-bootstrap v0.5.1 +ember-cli v1.13.13 +ember-cli-app-version v1.0.0 +ember-cli-babel v5.1.6 +ember-cli-content-security-policy v0.4.0 +ember-cli-dependency-checker v1.2.0 +ember-cli-htmlbars v1.0.2 +ember-cli-htmlbars-inline-precompile v0.3.1 +ember-cli-ic-ajax v0.2.1 +ember-cli-inject-live-reload v1.4.0 +ember-cli-jquery-ui v0.0.20 +ember-cli-qunit v1.2.1 +ember-cli-release v0.2.8 +ember-cli-shims v0.0.6 +ember-cli-sri v1.2.1 +ember-cli-test-loader v0.2.1 +ember-cli-uglify v1.2.0 +ember-d3 v0.1.0 +ember-data v2.1.0 +ember-disable-proxy-controllers v1.0.1 +ember-export-application-global v1.0.5 +ember-load-initializers v0.1.7 +ember-qunit v0.4.16 +ember-qunit-notifications v0.1.0 +ember-resolver v2.0.3 +ember-spin-spinner v0.2.3 +ember-truth-helpers v1.2.0 +jquery v2.1.4 +jquery-ui v1.11.4 +loader.js v3.3.0 +momentjs v2.10.6 +qunit v1.19.0 +select2 v4.0.0 +snippet-ss v1.11.0 +spin.js v2.3.2 +Azure Data Lake Store - Java client SDK 2.0.11 +JCodings 1.0.8 +Joni 2.1.2 +Mockito 1.8.5 +JUL to SLF4J bridge 1.7.25 +SLF4J API Module 1.7.25 +SLF4J LOG4J-12 Binding 1.7.25 +-------------------------------------------------------------------------------- + +The MIT License (MIT) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery +Apache HBase - Server which contains JQuery minified javascript library version 1.8.3 +Microsoft JDBC Driver for SQLServer - version 6.2.1.jre7 +-------------------------------------------------------------------------------- + +MIT License + +Copyright (c) 2003-2017 Optimatika + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +For: +oj! Algorithms - version 43.0 +-------------------------------------------------------------------------------- + +Copyright 2005, 2012, 2013 jQuery Foundation and other contributors, https://jquery.org/ + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/jquery/jquery + +The following license applies to all parts of this software except as +documented below: + +==== + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +==== + +All files located in the node_modules and external directories are +externally maintained libraries used by this software which have their +own licenses; we recommend you read them, as their terms may differ from +the terms above. + +For: +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js +-------------------------------------------------------------------------------- + +Copyright (c) 2014 Ivan Bozhanov + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js +-------------------------------------------------------------------------------- + +D3 is available under a 3-clause BSD license. For details, see: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE + +The binary distribution of this product bundles these dependencies under the +following license: +HSQLDB Database 2.3.4 +-------------------------------------------------------------------------------- +(HSQL License) +"COPYRIGHTS AND LICENSES (based on BSD License) + +For work developed by the HSQL Development Group: + +Copyright (c) 2001-2016, The HSQL Development Group +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +Neither the name of the HSQL Development Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +For work originally developed by the Hypersonic SQL Group: + +Copyright (c) 1995-2000 by the Hypersonic SQL Group. +All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +Neither the name of the Hypersonic SQL Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +This software consists of voluntary contributions made by many individuals on behalf of the +Hypersonic SQL Group." + +The binary distribution of this product bundles these dependencies under the +following license: +Java Servlet API 3.1.0 +servlet-api 2.5 +jsp-api 2.1 +jsr311-api 1.1.1 +Glassfish Jasper 6.1.14 +Servlet Specification 2.5 API 6.1.14 +-------------------------------------------------------------------------------- +(CDDL 1.0) +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 +1. Definitions.  + +1.1. Contributor means each individual or entity +that creates or contributes to the creation of +Modifications.  + +1.2. Contributor Version means the combination of the +Original Software, prior Modifications used by a Contributor (if any), and the +Modifications made by that particular Contributor.  + +1.3. Covered +Software means (a) the Original Software, or (b) Modifications, or (c) the +combination of files containing Original Software with files containing +Modifications, in each case including portions +thereof.  + +1.4. Executable means the Covered Software in any form other +than Source Code.  + +1.5. Initial Developer means the individual or entity +that first makes Original Software available under this +License.  + +1.6. Larger Work means a work which combines Covered Software or +portions thereof with code not governed by the terms of this +License.  + +1.7. License means this document.  + +1.8. Licensable means +having the right to grant, to the maximum extent possible, whether at the time +of the initial grant or subsequently acquired, any and all of the rights +conveyed herein.  + +1.9. Modifications means the Source Code and Executable +form of any of the following: +A. Any file that results from an addition to, +deletion from or modification of the contents of a file containing Original +Software or previous Modifications; +B. Any new file that contains any part of the Original Software +or previous Modification; or +C. Any new file that is contributed or otherwise made available +under the terms of this License.  + +1.10. Original Software means the Source Code and Executable form of +computer software code that is originally released under this License.  + +1.11. Patent Claims means any patent claim(s), now owned or +hereafter acquired, including without limitation, method, process, and apparatus +claims, in any patent Licensable by grantor.  + +1.12. Source Code means (a) the common form of computer software code in which +modifications are made and (b) associated documentation included in or +with such code.  + +1.13. You (or Your) means an individual or a legal entity exercising rights +under, and complying with all of the terms of, this License. For legal entities, +You includes any entity which controls, is controlled by, or is under common control +with You. For purposes of this definition, control means (a) the power, direct +or indirect, to cause the direction or management of such entity, whether by +contract or otherwise, or (b) ownership of more than fifty percent (50%) of the +outstanding shares or beneficial ownership of such entity.  + +2. License Grants. + +2.1. The Initial Developer Grant. Conditioned upon Your compliance +with Section 3.1 below and subject to third party intellectual property claims, +the Initial Developer hereby grants You a world-wide, royalty-free, +non-exclusive license:  + +(a) under intellectual property rights (other than +patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, +display, perform, sublicense and distribute the Original Software (or portions +thereof), with or without Modifications, and/or as part of a Larger Work; +and  + +(b) under Patent Claims infringed by the making, using or selling of +Original Software, to make, have made, use, practice, sell, and offer for sale, +and/or otherwise dispose of the Original Software (or portions +thereof); + +(c) The licenses granted in Sections 2.1(a) and (b) are +effective on the date Initial Developer first distributes or otherwise makes the +Original Software available to a third party under the terms of this +License; + +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or +(ii) the combination of the Original Software with other software or +devices.  + +2.2. Contributor Grant. Conditioned upon Your compliance with +Section 3.1 below and subject to third party intellectual property claims, each +Contributor hereby grants You a world-wide, royalty-free, non-exclusive +license:  + +(a) under intellectual property rights (other than patent or +trademark) Licensable by Contributor to use, reproduce, modify, display, +perform, sublicense and distribute the Modifications created by such Contributor +(or portions thereof), either on an unmodified basis, with other Modifications, +as Covered Software and/or as part of a Larger Work; and  + +(b) under Patent +Claims infringed by the making, using, or selling of Modifications made by that +Contributor either alone and/or in combination with its Contributor Version (or +portions of such combination), to make, use, sell, offer for sale, have made, +and/or otherwise dispose of: (1) Modifications made by that Contributor (or +portions thereof); and (2) the combination of Modifications made by that +Contributor with its Contributor Version (or portions of such +combination).  + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are +effective on the date Contributor first distributes or otherwise makes the +Modifications available to a third party. + +(d) Notwithstanding Section 2.2(b) +above, no patent license is granted: (1) for any code that Contributor has +deleted from the Contributor Version; (2) for infringements caused by: +(i) third party modifications of Contributor Version, or (ii) the combination +of Modifications made by that Contributor with other software (except as part of +the Contributor Version) or other devices; or (3) under Patent Claims infringed +by Covered Software in the absence of Modifications made by that +Contributor.  + +3. Distribution Obligations.  + +3.1. Availability of Source +Code. Any Covered Software that You distribute or otherwise make available in +Executable form must also be made available in Source Code form and that Source +Code form must be distributed only under the terms of this License. You must +include a copy of this License with every copy of the Source Code form of the +Covered Software You distribute or otherwise make available. You must inform +recipients of any such Covered Software in Executable form as to how they can +obtain such Covered Software in Source Code form in a reasonable manner on or +through a medium customarily used for software exchange.  + +3.2. +Modifications. The Modifications that You create or to which You contribute are +governed by the terms of this License. You represent that You believe Your +Modifications are Your original creation(s) and/or You have sufficient rights to +grant the rights conveyed by this License.  + +3.3. Required Notices. You must +include a notice in each of Your Modifications that identifies You as the +Contributor of the Modification. You may not remove or alter any copyright, +patent or trademark notices contained within the Covered Software, or any +notices of licensing or any descriptive text giving attribution to any +Contributor or the Initial Developer.  + +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source Code +form that alters or restricts the applicable version of this License or the +recipients rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. + +3.5. Distribution of Executable Versions. You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipients rights in the Source Code form from the rights set forth in this +License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer.  + +3.6. Larger Works. You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software.  + +4. Versions of the License.  + +4.1. +New Versions. Sun Microsystems, Inc. is the initial license steward and may +publish revised and/or new versions of this License from time to time. Each +version will be given a distinguishing version number. Except as provided in +Section 4.3, no one other than the license steward has the right to modify this +License.  + +4.2. Effect of New Versions. You may always continue to use, +distribute or otherwise make the Covered Software available under the terms of +the version of the License under which You originally received the Covered +Software. If the Initial Developer includes a notice in the Original Software +prohibiting it from being distributed or otherwise made available under any +subsequent version of the License, You must distribute and make the Covered +Software available under the terms of the version of the License under which You +originally received the Covered Software. Otherwise, You may also choose to use, +distribute or otherwise make the Covered Software available under the terms of +any subsequent version of the License published by the license +steward.  + +4.3. Modified Versions. When You are an Initial Developer and You +want to create a new license for Your Original Software, You may create and use +a modified version of this License if You: (a) rename the license and remove +any references to the name of the license steward (except to note that the +license differs from this License); and (b) otherwise make it clear that the +license contains terms which differ from this License.  + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, +WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT +LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, +MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY +COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER +OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR +CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS +LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER +THIS DISCLAIMER.  + +6. TERMINATION.  + +6.1. This License and the rights +granted hereunder will terminate automatically if You fail to comply with terms +herein and fail to cure such breach within 30 days of becoming aware of the +breach. Provisions which, by their nature, must remain in effect beyond the +termination of this License shall survive.  + +6.2. If You assert a patent +infringement claim (excluding declaratory judgment actions) against Initial +Developer or a Contributor (the Initial Developer or Contributor against whom +You assert such claim is referred to as Participant) alleging that the +Participant Software (meaning the Contributor Version where the Participant is a +Contributor or the Original Software where the Participant is the Initial +Developer) directly or indirectly infringes any patent, then any and all rights +granted directly or indirectly to You by such Participant, the Initial Developer +(if the Initial Developer is not the Participant) and all Contributors under +Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from +Participant terminate prospectively and automatically at the expiration of such +60 day notice period, unless if within such 60 day period You withdraw Your +claim with respect to the Participant Software against such Participant either +unilaterally or pursuant to a written agreement with Participant.  + +6.3. In +the event of termination under Sections 6.1 or 6.2 above, all end user licenses +that have been validly granted by You or any distributor hereunder prior to +termination (excluding licenses granted to You by any distributor) shall survive +termination.  + +7. LIMITATION OF LIABILITY. +UNDER NO CIRCUMSTANCES AND UNDER +NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, +SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF +COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY +PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY +CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF +GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER +COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE +POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO +LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO +THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT +ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO +THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.  + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a commercial item, as that term is defined in +48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as +that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer +software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. +1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through +227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software +with only those rights set forth herein. This U.S. Government Rights clause is +in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision +that addresses Government rights in computer software under this +License.  + +9. MISCELLANEOUS. +This License represents the complete agreement +concerning subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdictions conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software.  + +10. RESPONSIBILITY FOR CLAIMS. +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability.  + +The binary distribution of this product bundles these dependencies under the +following license: +jersey-client 1.19 +jersey-core 1.19 +jersey-grizzly2 1.19 +jersey-grizzly2-servlet 1.19 +jersey-json 1.19 +jersey-server 1.19 +jersey-servlet 1.19 +jersey-guice 1.19 +Jersey Test Framework - Grizzly 2 Module 1.19 +JAXB RI 2.2.3 +Java Architecture for XML Binding 2.2.11 +grizzly-framework 2.2.21 +grizzly-http 2.2.21 +grizzly-http-server 2.2.21 +grizzly-http-servlet 2.2.21 +grizzly-rcm 2.2.21 +-------------------------------------------------------------------------------- +(CDDL 1.1) +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 + +1. Definitions. + +1.1. “Contributor” means each individual or entity that creates or +contributes to the creation of Modifications. +1.2. “Contributor Version” means the combination of the Original Software, +prior Modifications used by a Contributor (if any), and the Modifications made +by that particular Contributor. +1.3. “Covered Software” means (a) the Original Software, or (b) +Modifications, or (c) the combination of files containing Original Software with +files containing Modifications, in each case including portions thereof. +1.4. “Executable” means the Covered Software in any form other than Source +Code. +1.5. “Initial Developer” means the individual or entity that first makes +Original Software available under this License. +1.6. “Larger Work” means a work which combines Covered Software or portions +thereof with code not governed by the terms of this License. +1.7. “License” means this document. +1.8. “Licensable” means having the right to grant, to the maximum extent +possible, whether at the time of the initial grant or subsequently acquired, any +and all of the rights conveyed herein. +1.9. “Modifications” means the Source Code and Executable form of any of the +following: +A. Any file that results from an addition to, deletion from or modification of +the contents of a file containing Original Software or previous Modifications; +B. Any new file that contains any part of the Original Software or previous +Modification; or +C. Any new file that is contributed or otherwise made available under the terms +of this License. +1.10. “Original Software” means the Source Code and Executable form of +computer software code that is originally released under this License. +1.11. “Patent Claims” means any patent claim(s), now owned or hereafter +acquired, including without limitation, method, process, and apparatus claims, +in any patent Licensable by grantor. +1.12. “Source Code” means (a) the common form of computer software code in +which modifications are made and (b) associated documentation included in or +with such code. +1.13. “You” (or “Your”) means an individual or a legal entity exercising +rights under, and complying with all of the terms of, this License. For legal +entities, “You” includes any entity which controls, is controlled by, or is +under common control with You. For purposes of this definition, “control” +means (a) the power, direct or indirect, to cause the direction or management of +such entity, whether by contract or otherwise, or (b) ownership of more than +fifty percent (50%) of the outstanding shares or beneficial ownership of such +entity. + +2. License Grants. + +2.1. The Initial Developer Grant. + +Conditioned upon Your compliance with Section 3.1 below and subject to +third party intellectual property claims, the Initial Developer hereby grants +You a world-wide, royalty-free, non-exclusive license: +(a) under intellectual +property rights (other than patent or trademark) Licensable by Initial +Developer, to use, reproduce, modify, display, perform, sublicense and +distribute the Original Software (or portions thereof), with or without +Modifications, and/or as part of a Larger Work; and +(b) under Patent Claims +infringed by the making, using or selling of Original Software, to make, have +made, use, practice, sell, and offer for sale, and/or otherwise dispose of the +Original Software (or portions thereof). +(c) The licenses granted in Sections +2.1(a) and (b) are effective on the date Initial Developer first distributes or +otherwise makes the Original Software available to a third party under the terms +of this License. +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or (ii) +the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. + +Conditioned upon Your compliance with Section 3.1 below and +subject to third party intellectual property claims, each Contributor hereby +grants You a world-wide, royalty-free, non-exclusive license: +(a) under +intellectual property rights (other than patent or trademark) Licensable by +Contributor to use, reproduce, modify, display, perform, sublicense and +distribute the Modifications created by such Contributor (or portions thereof), +either on an unmodified basis, with other Modifications, as Covered Software +and/or as part of a Larger Work; and +(b) under Patent Claims infringed by the +making, using, or selling of Modifications made by that Contributor either alone +and/or in combination with its Contributor Version (or portions of such +combination), to make, use, sell, offer for sale, have made, and/or otherwise +dispose of: (1) Modifications made by that Contributor (or portions thereof); +and (2) the combination of Modifications made by that Contributor with its +Contributor Version (or portions of such combination). +(c) The licenses granted +in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first +distributes or otherwise makes the Modifications available to a third +party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: +(1) for any code that Contributor has deleted from the Contributor Version; (2) +for infringements caused by: (i) third party modifications of Contributor +Version, or (ii) the combination of Modifications made by that Contributor with +other software (except as part of the Contributor Version) or other devices; or +(3) under Patent Claims infringed by Covered Software in the absence of +Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. +Any Covered Software that You distribute or +otherwise make available in Executable form must also be made available in +Source Code form and that Source Code form must be distributed only under the +terms of this License. You must include a copy of this License with every copy +of the Source Code form of the Covered Software You distribute or otherwise make +available. You must inform recipients of any such Covered Software in Executable +form as to how they can obtain such Covered Software in Source Code form in a +reasonable manner on or through a medium customarily used for software +exchange. +3.2. Modifications. +The Modifications that You create or to which +You contribute are governed by the terms of this License. You represent that You +believe Your Modifications are Your original creation(s) and/or You have +sufficient rights to grant the rights conveyed by this License. +3.3. Required Notices. +You must include a notice in each of Your Modifications that +identifies You as the Contributor of the Modification. You may not remove or +alter any copyright, patent or trademark notices contained within the Covered +Software, or any notices of licensing or any descriptive text giving attribution +to any Contributor or the Initial Developer. +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source +Code form that alters or restricts the applicable version of this License or the +recipients' rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. +3.5. Distribution of Executable Versions. +You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipient's rights in the Source Code form from the rights set forth in +this License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer. +3.6. Larger Works. +You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. +Oracle is the initial license steward and may publish revised and/or +new versions of this License from time to time. Each version will be given a +distinguishing version number. Except as provided in Section 4.3, no one other +than the license steward has the right to modify this License. +4.2. Effect of New Versions. +You may always continue to use, distribute or otherwise make the +Covered Software available under the terms of the version of the License under +which You originally received the Covered Software. If the Initial Developer +includes a notice in the Original Software prohibiting it from being distributed +or otherwise made available under any subsequent version of the License, You +must distribute and make the Covered Software available under the terms of the +version of the License under which You originally received the Covered Software. +Otherwise, You may also choose to use, distribute or otherwise make the Covered +Software available under the terms of any subsequent version of the License +published by the license steward. +4.3. Modified Versions. +When You are an +Initial Developer and You want to create a new license for Your Original +Software, You may create and use a modified version of this License if You: (a) +rename the license and remove any references to the name of the license steward +(except to note that the license differs from this License); and (b) otherwise +make it clear that the license contains terms which differ from this +License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS +LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE +IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR +NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED +SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY +RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE +COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF +WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED +SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will +terminate automatically if You fail to comply with terms herein and fail to cure +such breach within 30 days of becoming aware of the breach. Provisions which, by +their nature, must remain in effect beyond the termination of this License shall +survive. +6.2. If You assert a patent infringement claim (excluding declaratory +judgment actions) against Initial Developer or a Contributor (the Initial +Developer or Contributor against whom You assert such claim is referred to as +“Participant”) alleging that the Participant Software (meaning the +Contributor Version where the Participant is a Contributor or the Original +Software where the Participant is the Initial Developer) directly or indirectly +infringes any patent, then any and all rights granted directly or indirectly to +You by such Participant, the Initial Developer (if the Initial Developer is not +the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this +License shall, upon 60 days notice from Participant terminate prospectively and +automatically at the expiration of such 60 day notice period, unless if within +such 60 day period You withdraw Your claim with respect to the Participant +Software against such Participant either unilaterally or pursuant to a written +agreement with Participant. +6.3. If You assert a patent infringement claim +against Participant alleging that the Participant Software directly or +indirectly infringes any patent where such claim is resolved (such as by license +or settlement) prior to the initiation of patent infringement litigation, then +the reasonable value of the licenses granted by such Participant under Sections +2.1 or 2.2 shall be taken into account in determining the amount or value of any +payment or license. +6.4. In the event of termination under Sections 6.1 or 6.2 +above, all end user licenses that have been validly granted by You or any +distributor hereunder prior to termination (excluding licenses granted to You by +any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT +(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL +DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY +SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, +SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, +WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER +FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN +IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS +LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL +INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW +PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR +LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND +LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + +The Covered +Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 +(Oct. 1995), consisting of “commercial computer software” (as that term is +defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software +documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). +Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 +(June 1995), all U.S. Government End Users acquire Covered Software with only +those rights set forth herein. This U.S. Government Rights clause is in lieu of, +and supersedes, any other FAR, DFAR, or other clause or provision that addresses +Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning +subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdiction's conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys' fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability. + +The binary distribution of this product bundles these dependencies under the +following license: +Protocol Buffer Java API 2.5.0 +-------------------------------------------------------------------------------- +This license applies to all parts of Protocol Buffers except the following: + + - Atomicops support for generic gcc, located in + src/google/protobuf/stubs/atomicops_internals_generic_gcc.h. + This file is copyrighted by Red Hat Inc. + + - Atomicops support for AIX/POWER, located in + src/google/protobuf/stubs/atomicops_internals_power.h. + This file is copyrighted by Bloomberg Finance LP. + +Copyright 2014, Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +For: +XML Commons External Components XML APIs 1.3.04 +-------------------------------------------------------------------------------- +By obtaining, using and/or copying this work, you (the licensee) agree that you +have read, understood, and will comply with the following terms and conditions. + +Permission to copy, modify, and distribute this software and its documentation, +with or without modification, for any purpose and without fee or royalty is +hereby granted, provided that you include the following on ALL copies of the +software and documentation or portions thereof, including modifications: +- The full text of this NOTICE in a location viewable to users of the +redistributed or derivative work. +- Any pre-existing intellectual property disclaimers, notices, or terms and +conditions. If none exist, the W3C Software Short Notice should be included +(hypertext is preferred, text is permitted) within the body of any redistributed +or derivative code. +- Notice of any changes or modifications to the files, including the date changes +were made. (We recommend you provide URIs to the location from which the code is +derived.) + +The binary distribution of this product bundles these dependencies under the +following license: +JUnit 4.11 +Eclipse JDT Core 3.1.1 +-------------------------------------------------------------------------------- +(EPL v1.0) +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC +LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM +CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation +distributed under this Agreement, and +b) in the case of each subsequent Contributor: +i) changes to the Program, and +ii) additions to the Program; +where such changes and/or additions to the Program originate from and are +distributed by that particular Contributor. A Contribution 'originates' from a +Contributor if it was added to the Program by such Contributor itself or anyone +acting on such Contributor's behalf. Contributions do not include additions to +the Program which: (i) are separate modules of software distributed in +conjunction with the Program under their own license agreement, and (ii) are not +derivative works of the Program. +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are +necessarily infringed by the use or sale of its Contribution alone or when +combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, +including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free copyright license to +reproduce, prepare derivative works of, publicly display, publicly perform, +distribute and sublicense the Contribution of such Contributor, if any, and such +derivative works, in source code and object code form. +b) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed +Patents to make, use, sell, offer to sell, import and otherwise transfer the +Contribution of such Contributor, if any, in source code and object code form. +This patent license shall apply to the combination of the Contribution and the +Program if, at the time the Contribution is added by the Contributor, such +addition of the Contribution causes such combination to be covered by the +Licensed Patents. The patent license shall not apply to any other combinations +which include the Contribution. No hardware per se is licensed hereunder. +c) Recipient understands that although each Contributor grants the licenses to +its Contributions set forth herein, no assurances are provided by any +Contributor that the Program does not infringe the patent or other intellectual +property rights of any other entity. Each Contributor disclaims any liability to +Recipient for claims brought by any other entity based on infringement of +intellectual property rights or otherwise. As a condition to exercising the +rights and licenses granted hereunder, each Recipient hereby assumes sole +responsibility to secure any other intellectual property rights needed, if any. +For example, if a third party patent license is required to allow Recipient to +distribute the Program, it is Recipient's responsibility to acquire that license +before distributing the Program. +d) Each Contributor represents that to its knowledge it has sufficient copyright +rights in its Contribution, if any, to grant the copyright license set forth in +this Agreement. +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its +own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and +b) its license agreement: +i) effectively disclaims on behalf of all Contributors all warranties and +conditions, express and implied, including warranties or conditions of title and +non-infringement, and implied warranties or conditions of merchantability and +fitness for a particular purpose; +ii) effectively excludes on behalf of all Contributors all liability for +damages, including direct, indirect, special, incidental and consequential +damages, such as lost profits; +iii) states that any provisions which differ from this Agreement are offered by +that Contributor alone and not by any other party; and +iv) states that source code for the Program is available from such Contributor, +and informs licensees how to obtain it in a reasonable manner on or through a +medium customarily used for software exchange. +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and +b) a copy of this Agreement must be included with each copy of the Program. +Contributors may not remove or alter any copyright notices contained within the +Program. + +Each Contributor must identify itself as the originator of its Contribution, if +any, in a manner that reasonably allows subsequent Recipients to identify the +originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with +respect to end users, business partners and the like. While this license is +intended to facilitate the commercial use of the Program, the Contributor who +includes the Program in a commercial product offering should do so in a manner +which does not create potential liability for other Contributors. Therefore, if +a Contributor includes the Program in a commercial product offering, such +Contributor ("Commercial Contributor") hereby agrees to defend and indemnify +every other Contributor ("Indemnified Contributor") against any losses, damages +and costs (collectively "Losses") arising from claims, lawsuits and other legal +actions brought by a third party against the Indemnified Contributor to the +extent caused by the acts or omissions of such Commercial Contributor in +connection with its distribution of the Program in a commercial product +offering. The obligations in this section do not apply to any claims or Losses +relating to any actual or alleged intellectual property infringement. In order +to qualify, an Indemnified Contributor must: a) promptly notify the Commercial +Contributor in writing of such claim, and b) allow the Commercial Contributor to +control, and cooperate with the Commercial Contributor in, the defense and any +related settlement negotiations. The Indemnified Contributor may participate in +any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product +offering, Product X. That Contributor is then a Commercial Contributor. If that +Commercial Contributor then makes performance claims, or offers warranties +related to Product X, those performance claims and warranties are such +Commercial Contributor's responsibility alone. Under this section, the +Commercial Contributor would have to defend claims against the other +Contributors related to those performance claims and warranties, and if a court +requires any other Contributor to pay any damages as a result, the Commercial +Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR +IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, +NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each +Recipient is solely responsible for determining the appropriateness of using and +distributing the Program and assumes all risks associated with its exercise of +rights under this Agreement , including but not limited to the risks and costs +of program errors, compliance with applicable laws, damage to or loss of data, +programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY +CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST +PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS +GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this Agreement, and without further action by the parties hereto, such +provision shall be reformed to the minimum extent necessary to make such +provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a +cross-claim or counterclaim in a lawsuit) alleging that the Program itself +(excluding combinations of the Program with other software or hardware) +infringes such Recipient's patent(s), then such Recipient's rights granted under +Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to +comply with any of the material terms or conditions of this Agreement and does +not cure such failure in a reasonable period of time after becoming aware of +such noncompliance. If all Recipient's rights under this Agreement terminate, +Recipient agrees to cease use and distribution of the Program as soon as +reasonably practicable. However, Recipient's obligations under this Agreement +and any licenses granted by Recipient relating to the Program shall continue and +survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in +order to avoid inconsistency the Agreement is copyrighted and may only be +modified in the following manner. The Agreement Steward reserves the right to +publish new versions (including revisions) of this Agreement from time to time. +No one other than the Agreement Steward has the right to modify this Agreement. +The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation +may assign the responsibility to serve as the Agreement Steward to a suitable +separate entity. Each new version of the Agreement will be given a +distinguishing version number. The Program (including Contributions) may always +be distributed subject to the version of the Agreement under which it was +received. In addition, after a new version of the Agreement is published, +Contributor may elect to distribute the Program (including its Contributions) +under the new version. Except as expressly stated in Sections 2(a) and 2(b) +above, Recipient receives no rights or licenses to the intellectual property of +any Contributor under this Agreement, whether expressly, by implication, +estoppel or otherwise. All rights in the Program not expressly granted under +this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the +intellectual property laws of the United States of America. No party to this +Agreement will bring a legal action under this Agreement more than one year +after the cause of action arose. Each party waives its rights to a jury trial in +any resulting litigation. + +The binary distribution of this product bundles these dependencies under the +following license: +JSch 0.1.51 +ParaNamer Core 2.3 +JLine 0.9.94 +leveldbjni-all 1.8 +Hamcrest Core 1.3 +ASM Core 5.0.4 +ASM Commons 5.0.2 +ASM Tree 5.0.2 +-------------------------------------------------------------------------------- +(3-clause BSD) +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles these dependencies under the +following license: +FindBugs-jsr305 3.0.0 +dnsjava 2.1.7, Copyright (c) 1998-2011, Brian Wellington. All rights reserved. +-------------------------------------------------------------------------------- +(2-clause BSD) +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The views and conclusions contained in the software and documentation are those +of the authors and should not be interpreted as representing official policies, +either expressed or implied, of the FreeBSD Project. + +The binary distribution of this product bundles these dependencies under the +following license: +"Java Concurrency in Practice" book annotations 1.0 +-------------------------------------------------------------------------------- +(CCAL v2.5) +THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE COMMONS +PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY COPYRIGHT AND/OR +OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS AUTHORIZED UNDER THIS +LICENSE OR COPYRIGHT LAW IS PROHIBITED. + +BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE TO BE +BOUND BY THE TERMS OF THIS LICENSE. THE LICENSOR GRANTS YOU THE RIGHTS CONTAINED +HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND CONDITIONS. + +1. Definitions + +"Collective Work" means a work, such as a periodical issue, anthology or +encyclopedia, in which the Work in its entirety in unmodified form, along with a +number of other contributions, constituting separate and independent works in +themselves, are assembled into a collective whole. A work that constitutes a +Collective Work will not be considered a Derivative Work (as defined below) for +the purposes of this License. +"Derivative Work" means a work based upon the Work or upon the Work and other +pre-existing works, such as a translation, musical arrangement, dramatization, +fictionalization, motion picture version, sound recording, art reproduction, +abridgment, condensation, or any other form in which the Work may be recast, +transformed, or adapted, except that a work that constitutes a Collective Work +will not be considered a Derivative Work for the purpose of this License. For +the avoidance of doubt, where the Work is a musical composition or sound +recording, the synchronization of the Work in timed-relation with a moving image +("synching") will be considered a Derivative Work for the purpose of this +License. +"Licensor" means the individual or entity that offers the Work under the terms +of this License. +"Original Author" means the individual or entity who created the Work. +"Work" means the copyrightable work of authorship offered under the terms of +this License. +"You" means an individual or entity exercising rights under this License who has +not previously violated the terms of this License with respect to the Work, or +who has received express permission from the Licensor to exercise rights under +this License despite a previous violation. +2. Fair Use Rights. Nothing in this license is intended to reduce, limit, or +restrict any rights arising from fair use, first sale or other limitations on +the exclusive rights of the copyright owner under copyright law or other +applicable laws. + +3. License Grant. Subject to the terms and conditions of this License, Licensor +hereby grants You a worldwide, royalty-free, non-exclusive, perpetual (for the +duration of the applicable copyright) license to exercise the rights in the Work +as stated below: + +to reproduce the Work, to incorporate the Work into one or more Collective +Works, and to reproduce the Work as incorporated in the Collective Works; +to create and reproduce Derivative Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission the Work including as +incorporated in Collective Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission Derivative Works. +For the avoidance of doubt, where the work is a musical composition: + +Performance Royalties Under Blanket Licenses. Licensor waives the exclusive +right to collect, whether individually or via a performance rights society (e.g. +ASCAP, BMI, SESAC), royalties for the public performance or public digital +performance (e.g. webcast) of the Work. +Mechanical Rights and Statutory Royalties. Licensor waives the exclusive right +to collect, whether individually or via a music rights agency or designated +agent (e.g. Harry Fox Agency), royalties for any phonorecord You create from the +Work ("cover version") and distribute, subject to the compulsory license created +by 17 USC Section 115 of the US Copyright Act (or the equivalent in other +jurisdictions). +Webcasting Rights and Statutory Royalties. For the avoidance of doubt, where the +Work is a sound recording, Licensor waives the exclusive right to collect, +whether individually or via a performance-rights society (e.g. SoundExchange), +royalties for the public digital performance (e.g. webcast) of the Work, subject +to the compulsory license created by 17 USC Section 114 of the US Copyright Act +(or the equivalent in other jurisdictions). +The above rights may be exercised in all media and formats whether now known or +hereafter devised. The above rights include the right to make such modifications +as are technically necessary to exercise the rights in other media and formats. +All rights not expressly granted by Licensor are hereby reserved. + +4. Restrictions.The license granted in Section 3 above is expressly made subject +to and limited by the following restrictions: + +You may distribute, publicly display, publicly perform, or publicly digitally +perform the Work only under the terms of this License, and You must include a +copy of, or the Uniform Resource Identifier for, this License with every copy or +phonorecord of the Work You distribute, publicly display, publicly perform, or +publicly digitally perform. You may not offer or impose any terms on the Work +that alter or restrict the terms of this License or the recipients' exercise of +the rights granted hereunder. You may not sublicense the Work. You must keep +intact all notices that refer to this License and to the disclaimer of +warranties. You may not distribute, publicly display, publicly perform, or +publicly digitally perform the Work with any technological measures that control +access or use of the Work in a manner inconsistent with the terms of this +License Agreement. The above applies to the Work as incorporated in a Collective +Work, but this does not require the Collective Work apart from the Work itself +to be made subject to the terms of this License. If You create a Collective +Work, upon notice from any Licensor You must, to the extent practicable, remove +from the Collective Work any credit as required by clause 4(b), as requested. If +You create a Derivative Work, upon notice from any Licensor You must, to the +extent practicable, remove from the Derivative Work any credit as required by +clause 4(b), as requested. +If you distribute, publicly display, publicly perform, or publicly digitally +perform the Work or any Derivative Works or Collective Works, You must keep +intact all copyright notices for the Work and provide, reasonable to the medium +or means You are utilizing: (i) the name of the Original Author (or pseudonym, +if applicable) if supplied, and/or (ii) if the Original Author and/or Licensor +designate another party or parties (e.g. a sponsor institute, publishing entity, +journal) for attribution in Licensor's copyright notice, terms of service or by +other reasonable means, the name of such party or parties; the title of the Work +if supplied; to the extent reasonably practicable, the Uniform Resource +Identifier, if any, that Licensor specifies to be associated with the Work, +unless such URI does not refer to the copyright notice or licensing information +for the Work; and in the case of a Derivative Work, a credit identifying the use +of the Work in the Derivative Work (e.g., "French translation of the Work by +Original Author," or "Screenplay based on original Work by Original Author"). +Such credit may be implemented in any reasonable manner; provided, however, that +in the case of a Derivative Work or Collective Work, at a minimum such credit +will appear where any other comparable authorship credit appears and in a manner +at least as prominent as such other comparable authorship credit. +5. Representations, Warranties and Disclaimer + +UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR OFFERS +THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY KIND CONCERNING +THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE, INCLUDING, WITHOUT +LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY, FITNESS FOR A PARTICULAR +PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF LATENT OR OTHER DEFECTS, ACCURACY, +OR THE PRESENCE OF ABSENCE OF ERRORS, WHETHER OR NOT DISCOVERABLE. SOME +JURISDICTIONS DO NOT ALLOW THE EXCLUSION OF IMPLIED WARRANTIES, SO SUCH +EXCLUSION MAY NOT APPLY TO YOU. + +6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE LAW, IN +NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR ANY SPECIAL, +INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES ARISING OUT OF THIS +LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + +7. Termination + +This License and the rights granted hereunder will terminate automatically upon +any breach by You of the terms of this License. Individuals or entities who have +received Derivative Works or Collective Works from You under this License, +however, will not have their licenses terminated provided such individuals or +entities remain in full compliance with those licenses. Sections 1, 2, 5, 6, 7, +and 8 will survive any termination of this License. +Subject to the above terms and conditions, the license granted here is perpetual +(for the duration of the applicable copyright in the Work). Notwithstanding the +above, Licensor reserves the right to release the Work under different license +terms or to stop distributing the Work at any time; provided, however that any +such election will not serve to withdraw this License (or any other license that +has been, or is required to be, granted under the terms of this License), and +this License will continue in full force and effect unless terminated as stated +above. +8. Miscellaneous + +Each time You distribute or publicly digitally perform the Work or a Collective +Work, the Licensor offers to the recipient a license to the Work on the same +terms and conditions as the license granted to You under this License. +Each time You distribute or publicly digitally perform a Derivative Work, +Licensor offers to the recipient a license to the original Work on the same +terms and conditions as the license granted to You under this License. +If any provision of this License is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this License, and without further action by the parties to this +agreement, such provision shall be reformed to the minimum extent necessary to +make such provision valid and enforceable. +No term or provision of this License shall be deemed waived and no breach +consented to unless such waiver or consent shall be in writing and signed by the +party to be charged with such waiver or consent. +This License constitutes the entire agreement between the parties with respect +to the Work licensed here. There are no understandings, agreements or +representations with respect to the Work not specified here. Licensor shall not +be bound by any additional provisions that may appear in any communication from +You. This License may not be modified without the mutual written agreement of +the Licensor and You. + +The binary distribution of this product bundles these dependencies under the +following license: +jamon-runtime 2.4.1 +-------------------------------------------------------------------------------- +(MPL 2.0) + Mozilla Public License + Version 2.0 + +1. Definitions + +1.1. “Contributor” +means each individual or legal entity that creates, contributes to the creation +of, or owns Covered Software. + +1.2. “Contributor Version” +means the combination of the Contributions of others (if any) used by a +Contributor and that particular Contributor’s Contribution. + +1.3. “Contribution” +means Covered Software of a particular Contributor. + +1.4. “Covered Software” +means Source Code Form to which the initial Contributor has attached the notice +in Exhibit A, the Executable Form of such Source Code Form, and Modifications of +such Source Code Form, in each case including portions thereof. + +1.5. “Incompatible With Secondary Licenses” +means + +that the initial Contributor has attached the notice described in Exhibit B to +the Covered Software; or + +that the Covered Software was made available under the terms of version 1.1 or +earlier of the License, but not also under the terms of a Secondary License. + +1.6. “Executable Form” +means any form of the work other than Source Code Form. + +1.7. “Larger Work” +means a work that combines Covered Software with other material, in a separate +file or files, that is not Covered Software. + +1.8. “License” +means this document. + +1.9. “Licensable” +means having the right to grant, to the maximum extent possible, whether at the +time of the initial grant or subsequently, any and all of the rights conveyed by +this License. + +1.10. “Modifications” +means any of the following: + +any file in Source Code Form that results from an addition to, deletion from, or +modification of the contents of Covered Software; or + +any new file in Source Code Form that contains any Covered Software. + +1.11. “Patent Claims” of a Contributor +means any patent claim(s), including without limitation, method, process, and +apparatus claims, in any patent Licensable by such Contributor that would be +infringed, but for the grant of the License, by the making, using, selling, +offering for sale, having made, import, or transfer of either its Contributions +or its Contributor Version. + +1.12. “Secondary License” +means either the GNU General Public License, Version 2.0, the GNU Lesser General +Public License, Version 2.1, the GNU Affero General Public License, Version 3.0, +or any later versions of those licenses. + +1.13. “Source Code Form” +means the form of the work preferred for making modifications. + +1.14. “You” (or “Your”) +means an individual or a legal entity exercising rights under this License. For +legal entities, “You” includes any entity that controls, is controlled by, +or is under common control with You. For purposes of this definition, +“control” means (a) the power, direct or indirect, to cause the direction or +management of such entity, whether by contract or otherwise, or (b) ownership of +more than fifty percent (50%) of the outstanding shares or beneficial ownership +of such entity. + +2. License Grants and Conditions + +2.1. Grants + +Each Contributor hereby grants You a world-wide, royalty-free, non-exclusive +license: + +under intellectual property rights (other than patent or trademark) Licensable +by such Contributor to use, reproduce, make available, modify, display, perform, +distribute, and otherwise exploit its Contributions, either on an unmodified +basis, with Modifications, or as part of a Larger Work; and + +under Patent Claims of such Contributor to make, use, sell, offer for sale, have +made, import, and otherwise transfer either its Contributions or its Contributor +Version. + +2.2. Effective Date + +The licenses granted in Section 2.1 with respect to any Contribution become +effective for each Contribution on the date the Contributor first distributes +such Contribution. + +2.3. Limitations on Grant Scope + +The licenses granted in this Section 2 are the only rights granted under this +License. No additional rights or licenses will be implied from the distribution +or licensing of Covered Software under this License. Notwithstanding Section +2.1(b) above, no patent license is granted by a Contributor: + +for any code that a Contributor has removed from Covered Software; or + +for infringements caused by: (i) Your and any other third party’s +modifications of Covered Software, or (ii) the combination of its Contributions +with other software (except as part of its Contributor Version); or + +under Patent Claims infringed by Covered Software in the absence of its +Contributions. + +This License does not grant any rights in the trademarks, service marks, or +logos of any Contributor (except as may be necessary to comply with the notice +requirements in Section 3.4). + +2.4. Subsequent Licenses + +No Contributor makes additional grants as a result of Your choice to distribute +the Covered Software under a subsequent version of this License (see Section +10.2) or under the terms of a Secondary License (if permitted under the terms of +Section 3.3). + +2.5. Representation + +Each Contributor represents that the Contributor believes its Contributions are +its original creation(s) or it has sufficient rights to grant the rights to its +Contributions conveyed by this License. + +2.6. Fair Use + +This License is not intended to limit any rights You have under applicable +copyright doctrines of fair use, fair dealing, or other equivalents. + +2.7. Conditions + +Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted in +Section 2.1. + +3. Responsibilities + +3.1. Distribution of Source Form + +All distribution of Covered Software in Source Code Form, including any +Modifications that You create or to which You contribute, must be under the +terms of this License. You must inform recipients that the Source Code Form of +the Covered Software is governed by the terms of this License, and how they can +obtain a copy of this License. You may not attempt to alter or restrict the +recipients’ rights in the Source Code Form. + +3.2. Distribution of Executable Form + +If You distribute Covered Software in Executable Form then: + +such Covered Software must also be made available in Source Code Form, as +described in Section 3.1, and You must inform recipients of the Executable Form +how they can obtain a copy of such Source Code Form by reasonable means in a +timely manner, at a charge no more than the cost of distribution to the +recipient; and + +You may distribute such Executable Form under the terms of this License, or +sublicense it under different terms, provided that the license for the +Executable Form does not attempt to limit or alter the recipients’ rights in +the Source Code Form under this License. + +3.3. Distribution of a Larger Work + +You may create and distribute a Larger Work under terms of Your choice, provided +that You also comply with the requirements of this License for the Covered +Software. If the Larger Work is a combination of Covered Software with a work +governed by one or more Secondary Licenses, and the Covered Software is not +Incompatible With Secondary Licenses, this License permits You to additionally +distribute such Covered Software under the terms of such Secondary License(s), +so that the recipient of the Larger Work may, at their option, further +distribute the Covered Software under the terms of either this License or such +Secondary License(s). + +3.4. Notices + +You may not remove or alter the substance of any license notices (including +copyright notices, patent notices, disclaimers of warranty, or limitations of +liability) contained within the Source Code Form of the Covered Software, except +that You may alter any license notices to the extent required to remedy known +factual inaccuracies. + +3.5. Application of Additional Terms + +You may choose to offer, and to charge a fee for, warranty, support, indemnity +or liability obligations to one or more recipients of Covered Software. However, +You may do so only on Your own behalf, and not on behalf of any Contributor. You +must make it absolutely clear that any such warranty, support, indemnity, or +liability obligation is offered by You alone, and You hereby agree to indemnify +every Contributor for any liability incurred by such Contributor as a result of +warranty, support, indemnity or liability terms You offer. You may include +additional disclaimers of warranty and limitations of liability specific to any +jurisdiction. + +4. Inability to Comply Due to Statute or Regulation + +If it is impossible for You to comply with any of the terms of this License with +respect to some or all of the Covered Software due to statute, judicial order, +or regulation then You must: (a) comply with the terms of this License to the +maximum extent possible; and (b) describe the limitations and the code they +affect. Such description must be placed in a text file included with all +distributions of the Covered Software under this License. Except to the extent +prohibited by statute or regulation, such description must be sufficiently +detailed for a recipient of ordinary skill to be able to understand it. + +5. Termination + +5.1. The rights granted under this License will terminate automatically if You +fail to comply with any of its terms. However, if You become compliant, then the +rights granted under this License from a particular Contributor are reinstated +(a) provisionally, unless and until such Contributor explicitly and finally +terminates Your grants, and (b) on an ongoing basis, if such Contributor fails +to notify You of the non-compliance by some reasonable means prior to 60 days +after You have come back into compliance. Moreover, Your grants from a +particular Contributor are reinstated on an ongoing basis if such Contributor +notifies You of the non-compliance by some reasonable means, this is the first +time You have received notice of non-compliance with this License from such +Contributor, and You become compliant prior to 30 days after Your receipt of the +notice. + +5.2. If You initiate litigation against any entity by asserting a patent +infringement claim (excluding declaratory judgment actions, counter-claims, and +cross-claims) alleging that a Contributor Version directly or indirectly +infringes any patent, then the rights granted to You by any and all Contributors +for the Covered Software under Section 2.1 of this License shall terminate. + +5.3. In the event of termination under Sections 5.1 or 5.2 above, all end user +license agreements (excluding distributors and resellers) which have been +validly granted by You or Your distributors under this License prior to +termination shall survive termination. + +6. Disclaimer of Warranty + +Covered Software is provided under this License on an “as is” basis, without +warranty of any kind, either expressed, implied, or statutory, including, +without limitation, warranties that the Covered Software is free of defects, +merchantable, fit for a particular purpose or non-infringing. The entire risk as +to the quality and performance of the Covered Software is with You. Should any +Covered Software prove defective in any respect, You (not any Contributor) +assume the cost of any necessary servicing, repair, or correction. This +disclaimer of warranty constitutes an essential part of this License. No use of +any Covered Software is authorized under this License except under this +disclaimer. + +7. Limitation of Liability + +Under no circumstances and under no legal theory, whether tort (including +negligence), contract, or otherwise, shall any Contributor, or anyone who +distributes Covered Software as permitted above, be liable to You for any +direct, indirect, special, incidental, or consequential damages of any character +including, without limitation, damages for lost profits, loss of goodwill, work +stoppage, computer failure or malfunction, or any and all other commercial +damages or losses, even if such party shall have been informed of the +possibility of such damages. This limitation of liability shall not apply to +liability for death or personal injury resulting from such party’s negligence +to the extent applicable law prohibits such limitation. Some jurisdictions do +not allow the exclusion or limitation of incidental or consequential damages, so +this exclusion and limitation may not apply to You. + +8. Litigation + +Any litigation relating to this License may be brought only in the courts of a +jurisdiction where the defendant maintains its principal place of business and +such litigation shall be governed by laws of that jurisdiction, without +reference to its conflict-of-law provisions. Nothing in this Section shall +prevent a party’s ability to bring cross-claims or counter-claims. + +9. Miscellaneous + +This License represents the complete agreement concerning the subject matter +hereof. If any provision of this License is held to be unenforceable, such +provision shall be reformed only to the extent necessary to make it enforceable. +Any law or regulation which provides that the language of a contract shall be +construed against the drafter shall not be used to construe this License against +a Contributor. + +10. Versions of the License + +10.1. New Versions + +Mozilla Foundation is the license steward. Except as provided in Section 10.3, +no one other than the license steward has the right to modify or publish new +versions of this License. Each version will be given a distinguishing version +number. + +10.2. Effect of New Versions + +You may distribute the Covered Software under the terms of the version of the +License under which You originally received the Covered Software, or under the +terms of any subsequent version published by the license steward. + +10.3. Modified Versions + +If you create software not governed by this License, and you want to create a +new license for such software, you may create and use a modified version of this +License if you rename the license and remove any references to the name of the +license steward (except to note that such modified license differs from this +License). + +10.4. Distributing Source Code Form that is Incompatible With Secondary Licenses + +If You choose to distribute Source Code Form that is Incompatible With Secondary +Licenses under the terms of this version of the License, the notice described in +Exhibit B of this License must be attached. + +Exhibit A - Source Code Form License Notice + +This Source Code Form is subject to the terms of the Mozilla Public License, v. +2.0. If a copy of the MPL was not distributed with this file, You can obtain one +at https://mozilla.org/MPL/2.0/. + +If it is not possible or desirable to put the notice in a particular file, then +You may include the notice in a location (such as a LICENSE file in a relevant +directory) where a recipient would be likely to look for such a notice. + +You may add additional accurate notices of copyright ownership. + +Exhibit B - “Incompatible With Secondary Licenses” Notice + +This Source Code Form is “Incompatible With Secondary Licenses”, as defined +by the Mozilla Public License, v. 2.0. + +The binary distribution of this product bundles these dependencies under the +following license: +JDOM 1.1 +-------------------------------------------------------------------------------- +/*-- + + Copyright (C) 2000-2004 Jason Hunter & Brett McLaughlin. + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions, and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions, and the disclaimer that follows + these conditions in the documentation and/or other materials + provided with the distribution. + + 3. The name "JDOM" must not be used to endorse or promote products + derived from this software without prior written permission. For + written permission, please contact . + + 4. Products derived from this software may not be called "JDOM", nor + may "JDOM" appear in their name, without prior written permission + from the JDOM Project Management . + + In addition, we request (but do not require) that you include in the + end-user documentation provided with the redistribution and/or in the + software itself an acknowledgement equivalent to the following: + "This product includes software developed by the + JDOM Project (http://www.jdom.org/)." + Alternatively, the acknowledgment may be graphical using the logos + available at http://www.jdom.org/images/logos. + + THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED + WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE JDOM AUTHORS OR THE PROJECT + CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF + USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT + OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + SUCH DAMAGE. + + This software consists of voluntary contributions made by many + individuals on behalf of the JDOM Project and was originally + created by Jason Hunter and + Brett McLaughlin . For more information + on the JDOM Project, please see . + + */ + +The binary distribution of this product bundles these dependencies under the +following license: +Hbase Server 1.2.4 +-------------------------------------------------------------------------------- +This project bundles a derivative image for our Orca Logo. This image is +available under the Creative Commons By Attribution 3.0 License. + + Creative Commons Legal Code + + Attribution 3.0 Unported + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS LICENSE DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE INFORMATION PROVIDED, AND DISCLAIMS LIABILITY FOR + DAMAGES RESULTING FROM ITS USE. + + License + + THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE + COMMONS PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY + COPYRIGHT AND/OR OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS + AUTHORIZED UNDER THIS LICENSE OR COPYRIGHT LAW IS PROHIBITED. + + BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE + TO BE BOUND BY THE TERMS OF THIS LICENSE. TO THE EXTENT THIS LICENSE MAY + BE CONSIDERED TO BE A CONTRACT, THE LICENSOR GRANTS YOU THE RIGHTS + CONTAINED HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND + CONDITIONS. + + 1. Definitions + + a. "Adaptation" means a work based upon the Work, or upon the Work and + other pre-existing works, such as a translation, adaptation, + derivative work, arrangement of music or other alterations of a + literary or artistic work, or phonogram or performance and includes + cinematographic adaptations or any other form in which the Work may be + recast, transformed, or adapted including in any form recognizably + derived from the original, except that a work that constitutes a + Collection will not be considered an Adaptation for the purpose of + this License. For the avoidance of doubt, where the Work is a musical + work, performance or phonogram, the synchronization of the Work in + timed-relation with a moving image ("synching") will be considered an + Adaptation for the purpose of this License. + b. "Collection" means a collection of literary or artistic works, such as + encyclopedias and anthologies, or performances, phonograms or + broadcasts, or other works or subject matter other than works listed + in Section 1(f) below, which, by reason of the selection and + arrangement of their contents, constitute intellectual creations, in + which the Work is included in its entirety in unmodified form along + with one or more other contributions, each constituting separate and + independent works in themselves, which together are assembled into a + collective whole. A work that constitutes a Collection will not be + considered an Adaptation (as defined above) for the purposes of this + License. + c. "Distribute" means to make available to the public the original and + copies of the Work or Adaptation, as appropriate, through sale or + other transfer of ownership. + d. "Licensor" means the individual, individuals, entity or entities that + offer(s) the Work under the terms of this License. + e. "Original Author" means, in the case of a literary or artistic work, + the individual, individuals, entity or entities who created the Work + or if no individual or entity can be identified, the publisher; and in + addition (i) in the case of a performance the actors, singers, + musicians, dancers, and other persons who act, sing, deliver, declaim, + play in, interpret or otherwise perform literary or artistic works or + expressions of folklore; (ii) in the case of a phonogram the producer + being the person or legal entity who first fixes the sounds of a + performance or other sounds; and, (iii) in the case of broadcasts, the + organization that transmits the broadcast. + f. "Work" means the literary and/or artistic work offered under the terms + of this License including without limitation any production in the + literary, scientific and artistic domain, whatever may be the mode or + form of its expression including digital form, such as a book, + pamphlet and other writing; a lecture, address, sermon or other work + of the same nature; a dramatic or dramatico-musical work; a + choreographic work or entertainment in dumb show; a musical + composition with or without words; a cinematographic work to which are + assimilated works expressed by a process analogous to cinematography; + a work of drawing, painting, architecture, sculpture, engraving or + lithography; a photographic work to which are assimilated works + expressed by a process analogous to photography; a work of applied + art; an illustration, map, plan, sketch or three-dimensional work + relative to geography, topography, architecture or science; a + performance; a broadcast; a phonogram; a compilation of data to the + extent it is protected as a copyrightable work; or a work performed by + a variety or circus performer to the extent it is not otherwise + considered a literary or artistic work. + g. "You" means an individual or entity exercising rights under this + License who has not previously violated the terms of this License with + respect to the Work, or who has received express permission from the + Licensor to exercise rights under this License despite a previous + violation. + h. "Publicly Perform" means to perform public recitations of the Work and + to communicate to the public those public recitations, by any means or + process, including by wire or wireless means or public digital + performances; to make available to the public Works in such a way that + members of the public may access these Works from a place and at a + place individually chosen by them; to perform the Work to the public + by any means or process and the communication to the public of the + performances of the Work, including by public digital performance; to + broadcast and rebroadcast the Work by any means including signs, + sounds or images. + i. "Reproduce" means to make copies of the Work by any means including + without limitation by sound or visual recordings and the right of + fixation and reproducing fixations of the Work, including storage of a + protected performance or phonogram in digital form or other electronic + medium. + + 2. Fair Dealing Rights. Nothing in this License is intended to reduce, + limit, or restrict any uses free from copyright or rights arising from + limitations or exceptions that are provided for in connection with the + copyright protection under copyright law or other applicable laws. + + 3. License Grant. Subject to the terms and conditions of this License, + Licensor hereby grants You a worldwide, royalty-free, non-exclusive, + perpetual (for the duration of the applicable copyright) license to + exercise the rights in the Work as stated below: + + a. to Reproduce the Work, to incorporate the Work into one or more + Collections, and to Reproduce the Work as incorporated in the + Collections; + b. to create and Reproduce Adaptations provided that any such Adaptation, + including any translation in any medium, takes reasonable steps to + clearly label, demarcate or otherwise identify that changes were made + to the original Work. For example, a translation could be marked "The + original work was translated from English to Spanish," or a + modification could indicate "The original work has been modified."; + c. to Distribute and Publicly Perform the Work including as incorporated + in Collections; and, + d. to Distribute and Publicly Perform Adaptations. + e. For the avoidance of doubt: + + i. Non-waivable Compulsory License Schemes. In those jurisdictions in + which the right to collect royalties through any statutory or + compulsory licensing scheme cannot be waived, the Licensor + reserves the exclusive right to collect such royalties for any + exercise by You of the rights granted under this License; + ii. Waivable Compulsory License Schemes. In those jurisdictions in + which the right to collect royalties through any statutory or + compulsory licensing scheme can be waived, the Licensor waives the + exclusive right to collect such royalties for any exercise by You + of the rights granted under this License; and, + iii. Voluntary License Schemes. The Licensor waives the right to + collect royalties, whether individually or, in the event that the + Licensor is a member of a collecting society that administers + voluntary licensing schemes, via that society, from any exercise + by You of the rights granted under this License. + + The above rights may be exercised in all media and formats whether now + known or hereafter devised. The above rights include the right to make + such modifications as are technically necessary to exercise the rights in + other media and formats. Subject to Section 8(f), all rights not expressly + granted by Licensor are hereby reserved. + + 4. Restrictions. The license granted in Section 3 above is expressly made + subject to and limited by the following restrictions: + + a. You may Distribute or Publicly Perform the Work only under the terms + of this License. You must include a copy of, or the Uniform Resource + Identifier (URI) for, this License with every copy of the Work You + Distribute or Publicly Perform. You may not offer or impose any terms + on the Work that restrict the terms of this License or the ability of + the recipient of the Work to exercise the rights granted to that + recipient under the terms of the License. You may not sublicense the + Work. You must keep intact all notices that refer to this License and + to the disclaimer of warranties with every copy of the Work You + Distribute or Publicly Perform. When You Distribute or Publicly + Perform the Work, You may not impose any effective technological + measures on the Work that restrict the ability of a recipient of the + Work from You to exercise the rights granted to that recipient under + the terms of the License. This Section 4(a) applies to the Work as + incorporated in a Collection, but this does not require the Collection + apart from the Work itself to be made subject to the terms of this + License. If You create a Collection, upon notice from any Licensor You + must, to the extent practicable, remove from the Collection any credit + as required by Section 4(b), as requested. If You create an + Adaptation, upon notice from any Licensor You must, to the extent + practicable, remove from the Adaptation any credit as required by + Section 4(b), as requested. + b. If You Distribute, or Publicly Perform the Work or any Adaptations or + Collections, You must, unless a request has been made pursuant to + Section 4(a), keep intact all copyright notices for the Work and + provide, reasonable to the medium or means You are utilizing: (i) the + name of the Original Author (or pseudonym, if applicable) if supplied, + and/or if the Original Author and/or Licensor designate another party + or parties (e.g., a sponsor institute, publishing entity, journal) for + attribution ("Attribution Parties") in Licensor's copyright notice, + terms of service or by other reasonable means, the name of such party + or parties; (ii) the title of the Work if supplied; (iii) to the + extent reasonably practicable, the URI, if any, that Licensor + specifies to be associated with the Work, unless such URI does not + refer to the copyright notice or licensing information for the Work; + and (iv) , consistent with Section 3(b), in the case of an Adaptation, + a credit identifying the use of the Work in the Adaptation (e.g., + "French translation of the Work by Original Author," or "Screenplay + based on original Work by Original Author"). The credit required by + this Section 4 (b) may be implemented in any reasonable manner; + provided, however, that in the case of a Adaptation or Collection, at + a minimum such credit will appear, if a credit for all contributing + authors of the Adaptation or Collection appears, then as part of these + credits and in a manner at least as prominent as the credits for the + other contributing authors. For the avoidance of doubt, You may only + use the credit required by this Section for the purpose of attribution + in the manner set out above and, by exercising Your rights under this + License, You may not implicitly or explicitly assert or imply any + connection with, sponsorship or endorsement by the Original Author, + Licensor and/or Attribution Parties, as appropriate, of You or Your + use of the Work, without the separate, express prior written + permission of the Original Author, Licensor and/or Attribution + Parties. + c. Except as otherwise agreed in writing by the Licensor or as may be + otherwise permitted by applicable law, if You Reproduce, Distribute or + Publicly Perform the Work either by itself or as part of any + Adaptations or Collections, You must not distort, mutilate, modify or + take other derogatory action in relation to the Work which would be + prejudicial to the Original Author's honor or reputation. Licensor + agrees that in those jurisdictions (e.g. Japan), in which any exercise + of the right granted in Section 3(b) of this License (the right to + make Adaptations) would be deemed to be a distortion, mutilation, + modification or other derogatory action prejudicial to the Original + Author's honor and reputation, the Licensor will waive or not assert, + as appropriate, this Section, to the fullest extent permitted by the + applicable national law, to enable You to reasonably exercise Your + right under Section 3(b) of this License (right to make Adaptations) + but not otherwise. + + 5. Representations, Warranties and Disclaimer + + UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR + OFFERS THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY + KIND CONCERNING THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE, + INCLUDING, WITHOUT LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY, + FITNESS FOR A PARTICULAR PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF + LATENT OR OTHER DEFECTS, ACCURACY, OR THE PRESENCE OF ABSENCE OF ERRORS, + WHETHER OR NOT DISCOVERABLE. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION + OF IMPLIED WARRANTIES, SO SUCH EXCLUSION MAY NOT APPLY TO YOU. + + 6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE + LAW, IN NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR + ANY SPECIAL, INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES + ARISING OUT OF THIS LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS + BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + + 7. Termination + + a. This License and the rights granted hereunder will terminate + automatically upon any breach by You of the terms of this License. + Individuals or entities who have received Adaptations or Collections + from You under this License, however, will not have their licenses + terminated provided such individuals or entities remain in full + compliance with those licenses. Sections 1, 2, 5, 6, 7, and 8 will + survive any termination of this License. + b. Subject to the above terms and conditions, the license granted here is + perpetual (for the duration of the applicable copyright in the Work). + Notwithstanding the above, Licensor reserves the right to release the + Work under different license terms or to stop distributing the Work at + any time; provided, however that any such election will not serve to + withdraw this License (or any other license that has been, or is + required to be, granted under the terms of this License), and this + License will continue in full force and effect unless terminated as + stated above. + + 8. Miscellaneous + + a. Each time You Distribute or Publicly Perform the Work or a Collection, + the Licensor offers to the recipient a license to the Work on the same + terms and conditions as the license granted to You under this License. + b. Each time You Distribute or Publicly Perform an Adaptation, Licensor + offers to the recipient a license to the original Work on the same + terms and conditions as the license granted to You under this License. + c. If any provision of this License is invalid or unenforceable under + applicable law, it shall not affect the validity or enforceability of + the remainder of the terms of this License, and without further action + by the parties to this agreement, such provision shall be reformed to + the minimum extent necessary to make such provision valid and + enforceable. + d. No term or provision of this License shall be deemed waived and no + breach consented to unless such waiver or consent shall be in writing + and signed by the party to be charged with such waiver or consent. + e. This License constitutes the entire agreement between the parties with + respect to the Work licensed here. There are no understandings, + agreements or representations with respect to the Work not specified + here. Licensor shall not be bound by any additional provisions that + may appear in any communication from You. This License may not be + modified without the mutual written agreement of the Licensor and You. + f. The rights granted under, and the subject matter referenced, in this + License were drafted utilizing the terminology of the Berne Convention + for the Protection of Literary and Artistic Works (as amended on + September 28, 1979), the Rome Convention of 1961, the WIPO Copyright + Treaty of 1996, the WIPO Performances and Phonograms Treaty of 1996 + and the Universal Copyright Convention (as revised on July 24, 1971). + These rights and subject matter take effect in the relevant + jurisdiction in which the License terms are sought to be enforced + according to the corresponding provisions of the implementation of + those treaty provisions in the applicable national law. If the + standard suite of rights granted under applicable copyright law + includes additional rights not granted under this License, such + additional rights are deemed to be included in the License; this + License is not intended to restrict the license of any rights under + applicable law. + + + Creative Commons Notice + + Creative Commons is not a party to this License, and makes no warranty + whatsoever in connection with the Work. Creative Commons will not be + liable to You or any party on any legal theory for any damages + whatsoever, including without limitation any general, special, + incidental or consequential damages arising in connection to this + license. Notwithstanding the foregoing two (2) sentences, if Creative + Commons has expressly identified itself as the Licensor hereunder, it + shall have all rights and obligations of Licensor. + + Except for the limited purpose of indicating to the public that the + Work is licensed under the CCPL, Creative Commons does not authorize + the use by either party of the trademark "Creative Commons" or any + related trademark or logo of Creative Commons without the prior + written consent of Creative Commons. Any permitted use will be in + compliance with Creative Commons' then-current trademark usage + guidelines, as may be published on its website or otherwise made + available upon request from time to time. For the avoidance of doubt, + this trademark restriction does not form part of this License. + + Creative Commons may be contacted at https://creativecommons.org/. +-------------------------------------------------------------------------------- + +For: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs +/server/datanode/checker/AbstractFuture.java and +hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs +/server/datanode/checker/TimeoutFuture.java + +Copyright (C) 2007 The Guava Authors + +Licensed 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. diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-re2j b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-re2j new file mode 100644 index 0000000000000..b620ae68fe332 --- /dev/null +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-re2j @@ -0,0 +1,32 @@ +This is a work derived from Russ Cox's RE2 in Go, whose license +http://golang.org/LICENSE is as follows: + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-stax2api b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-stax2api new file mode 100644 index 0000000000000..0ed6361699658 --- /dev/null +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/licenses/LICENSE-stax2api @@ -0,0 +1,22 @@ +Copyright woodstox stax2api contributors. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, +INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, +BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, +WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-filesystems/flink-s3-fs-base/pom.xml b/flink-filesystems/flink-s3-fs-base/pom.xml index 6cc877dd57305..30fceadd57832 100644 --- a/flink-filesystems/flink-s3-fs-base/pom.xml +++ b/flink-filesystems/flink-s3-fs-base/pom.xml @@ -177,6 +177,7 @@ under the License. mime.types mozilla/** META-INF/maven/** + META-INF/LICENSE.txt diff --git a/flink-filesystems/flink-s3-fs-base/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-base/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..eed06551bedec --- /dev/null +++ b/flink-filesystems/flink-s3-fs-base/src/main/resources/META-INF/NOTICE @@ -0,0 +1,24 @@ +flink-s3-fs-base +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hadoop:hadoop-aws:3.1.0 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpclient:4.5.3 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- com.amazonaws:aws-java-sdk-core:1.11.271 +- com.amazonaws:aws-java-sdk-dynamodb:1.11.271 +- com.amazonaws:aws-java-sdk-kms:1.11.271 +- com.amazonaws:aws-java-sdk-s3:1.11.271 +- com.amazonaws:jmespath-java:1.11.271 +- software.amazon.ion:ion-java:1.0.2 +- com.fasterxml.jackson.core:jackson-annotations:2.6.0 +- com.fasterxml.jackson.core:jackson-core:2.6.7 +- com.fasterxml.jackson.core:jackson-databind:2.6.7.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.6.7 +- joda-time:joda-time:2.5 diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index cf35a04a585b2..6560c320976b0 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -269,6 +269,7 @@ under the License. META-INF/maven/io.airlift/** META-INF/maven/com*/** META-INF/maven/org.apache.flink/force-shading/** + META-INF/LICENSE.txt diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..b81e974b80e39 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE @@ -0,0 +1,26 @@ +flink-s3-fs-presto +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.facebook.presto:presto-hive:0.185 +- com.facebook.presto.hadoop:hadoop-apache2:2.7.3-1 +- com.google.guava:guava:21.0 +- io.airlift:configuration:0.148 +- io.airlift:log:0.148 +- io.airlift:stats:0.148 +- io.airlift:units:1.0 +- io.airlift:slice:0.31 +- com.fasterxml.jackson.core:jackson-annotations:2.8.1 +- com.fasterxml.jackson.core:jackson-core:2.8.1 +- com.fasterxml.jackson.core:jackson-databind:2.8.1 +- joda-time:joda-time:2.5 +- org.weakref:jmxutils:1.19 + +This project bundles the following dependencies under the Creative Commons CC0 1.0 Universal Public Domain Dedication License (http://creativecommons.org/publicdomain/zero/1.0/) +See bundled license files for details. + +- org.hdrhistogram:HdrHistogram:2.1.9 diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/licenses/LICENSE-hdrhistogram b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/licenses/LICENSE-hdrhistogram new file mode 100644 index 0000000000000..09c38ea51608e --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/licenses/LICENSE-hdrhistogram @@ -0,0 +1,125 @@ +The code was Written by Gil Tene, Michael Barker, and Matt Warren, +and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ + +Creative Commons Legal Code + +CC0 1.0 Universal + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS + PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM + THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED + HEREUNDER. + +Statement of Purpose + +The laws of most jurisdictions throughout the world automatically confer +exclusive Copyright and Related Rights (defined below) upon the creator +and subsequent owner(s) (each and all, an "owner") of an original work of +authorship and/or a database (each, a "Work"). + +Certain owners wish to permanently relinquish those rights to a Work for +the purpose of contributing to a commons of creative, cultural and +scientific works ("Commons") that the public can reliably and without fear +of later claims of infringement build upon, modify, incorporate in other +works, reuse and redistribute as freely as possible in any form whatsoever +and for any purposes, including without limitation commercial purposes. +These owners may contribute to the Commons to promote the ideal of a free +culture and the further production of creative, cultural and scientific +works, or to gain reputation or greater distribution for their Work in +part through the use and efforts of others. + +For these and/or other purposes and motivations, and without any +expectation of additional consideration or compensation, the person +associating CC0 with a Work (the "Affirmer"), to the extent that he or she +is an owner of Copyright and Related Rights in the Work, voluntarily +elects to apply CC0 to the Work and publicly distribute the Work under its +terms, with knowledge of his or her Copyright and Related Rights in the +Work and the meaning and intended legal effect of CC0 on those rights. + +1. Copyright and Related Rights. A Work made available under CC0 may be +protected by copyright and related or neighboring rights ("Copyright and +Related Rights"). Copyright and Related Rights include, but are not +limited to, the following: + + i. the right to reproduce, adapt, distribute, perform, display, + communicate, and translate a Work; + ii. moral rights retained by the original author(s) and/or performer(s); +iii. publicity and privacy rights pertaining to a person's image or + likeness depicted in a Work; + iv. rights protecting against unfair competition in regards to a Work, + subject to the limitations in paragraph 4(a), below; + v. rights protecting the extraction, dissemination, use and reuse of data + in a Work; + vi. database rights (such as those arising under Directive 96/9/EC of the + European Parliament and of the Council of 11 March 1996 on the legal + protection of databases, and under any national implementation + thereof, including any amended or successor version of such + directive); and +vii. other similar, equivalent or corresponding rights throughout the + world based on applicable law or treaty, and any national + implementations thereof. + +2. Waiver. To the greatest extent permitted by, but not in contravention +of, applicable law, Affirmer hereby overtly, fully, permanently, +irrevocably and unconditionally waives, abandons, and surrenders all of +Affirmer's Copyright and Related Rights and associated claims and causes +of action, whether now known or unknown (including existing as well as +future claims and causes of action), in the Work (i) in all territories +worldwide, (ii) for the maximum duration provided by applicable law or +treaty (including future time extensions), (iii) in any current or future +medium and for any number of copies, and (iv) for any purpose whatsoever, +including without limitation commercial, advertising or promotional +purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each +member of the public at large and to the detriment of Affirmer's heirs and +successors, fully intending that such Waiver shall not be subject to +revocation, rescission, cancellation, termination, or any other legal or +equitable action to disrupt the quiet enjoyment of the Work by the public +as contemplated by Affirmer's express Statement of Purpose. + +3. Public License Fallback. Should any part of the Waiver for any reason +be judged legally invalid or ineffective under applicable law, then the +Waiver shall be preserved to the maximum extent permitted taking into +account Affirmer's express Statement of Purpose. In addition, to the +extent the Waiver is so judged Affirmer hereby grants to each affected +person a royalty-free, non transferable, non sublicensable, non exclusive, +irrevocable and unconditional license to exercise Affirmer's Copyright and +Related Rights in the Work (i) in all territories worldwide, (ii) for the +maximum duration provided by applicable law or treaty (including future +time extensions), (iii) in any current or future medium and for any number +of copies, and (iv) for any purpose whatsoever, including without +limitation commercial, advertising or promotional purposes (the +"License"). The License shall be deemed effective as of the date CC0 was +applied by Affirmer to the Work. Should any part of the License for any +reason be judged legally invalid or ineffective under applicable law, such +partial invalidity or ineffectiveness shall not invalidate the remainder +of the License, and in such case Affirmer hereby affirms that he or she +will not (i) exercise any of his or her remaining Copyright and Related +Rights in the Work or (ii) assert any associated claims and causes of +action with respect to the Work, in either case contrary to Affirmer's +express Statement of Purpose. + +4. Limitations and Disclaimers. + + a. No trademark or patent rights held by Affirmer are waived, abandoned, + surrendered, licensed or otherwise affected by this document. + b. Affirmer offers the Work as-is and makes no representations or + warranties of any kind concerning the Work, express, implied, + statutory or otherwise, including without limitation warranties of + title, merchantability, fitness for a particular purpose, non + infringement, or the absence of latent or other defects, accuracy, or + the present or absence of errors, whether or not discoverable, all to + the greatest extent permissible under applicable law. + c. Affirmer disclaims responsibility for clearing rights of other persons + that may apply to the Work or any use thereof, including without + limitation any person's Copyright and Related Rights in the Work. + Further, Affirmer disclaims responsibility for obtaining any necessary + consents, permissions or other rights required for any use of the + Work. + d. Affirmer understands and acknowledges that Creative Commons is not a + party to this document and has no duty or obligation with respect to + this CC0 or use of the Work. From e80b6087c4e46c04e5fee4478da0ad4abf0781c7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 23 Nov 2018 17:49:43 +0100 Subject: [PATCH 164/359] [FLINK-10987] Add licenses and notice for flink-swift-fs-hadoop --- .../flink-swift-fs-hadoop/pom.xml | 3 + .../src/main/resources/META-INF/NOTICE | 1655 +++++++++++++++++ .../META-INF/licenses/LICENSE-protobuf | 36 + .../META-INF/licenses/LICENSE-xmlenc | 28 + 4 files changed, 1722 insertions(+) create mode 100644 flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE create mode 100644 flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-protobuf create mode 100644 flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-xmlenc diff --git a/flink-filesystems/flink-swift-fs-hadoop/pom.xml b/flink-filesystems/flink-swift-fs-hadoop/pom.xml index d0445d659ceee..cd6ff253de823 100644 --- a/flink-filesystems/flink-swift-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-swift-fs-hadoop/pom.xml @@ -247,6 +247,9 @@ under the License. META-INF/maven/org.apache.commons/** META-INF/maven/org.apache.flink/flink-hadoop-fs/** META-INF/maven/org.apache.flink/force-shading/** + META-INF/LICENSE.txt + META-INF/ASL2.0 + META-INF/README.txt core-default.xml diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..1d4bcc1601f62 --- /dev/null +++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -0,0 +1,1655 @@ +flink-swift-fs-hadoop +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hadoop:hadoop-annotations:2.8.1 +- org.apache.hadoop:hadoop-auth:2.8.1 +- org.apache.hadoop:hadoop-client:2.8.1 +- org.apache.hadoop:hadoop-common:2.8.1 +- org.apache.hadoop:hadoop-hdfs:2.8.1 +- org.apache.hadoop:hadoop-hdfs-client:2.8.1 +- org.apache.hadoop:hadoop-openstack:2.8.1 +- org.apache.htrace:htrace-core4:4.0.1-incubating +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-math3:3.5 +- commons-beanutils:commons-beanutils:1.8.3 +- commons-cli:commons-cli:1.3.1 +- commons-codec:commons-codec:1.10 +- commons-collections:commons-collections:3.2.2 +- commons-configuration:commons-configuration:1.7 +- commons-digester:commons-digester:1.8.1 +- commons-io:commons-io:2.4 +- commons-lang:commons-lang:2.6 +- commons-logging:commons-logging:1.1.3 +- commons-net:commons-net:3.1 +- commons-httpclient:commons-httpclient:3.1 +- com.google.guava:guava:11.0.2 +- com.google.code.gson:gson:2.2.4 +- com.squareup.okhttp:okhttp:2.4.0 +- com.squareup.okio:okio:1.4.0 +- net.minidev:json-smart:1.1.1 +- com.nimbusds:nimbus-jose-jwt:3.9 +- org.mortbay.jetty:jetty-sslengine:6.1.26 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 + +This project bundles the following dependencies under the BSD License. +See bundled license files for details. + +- xmlenc:xmlenc:0.52 +- com.google.protobuf:protobuf-java:2.5.0 + +This project bundles "net.jcip:jcip-annotations:1.0". +Written by Brian Goetz and Tim Peierls with assistance from members of JCP JSR-166 Expert Group +and released to the public domain, as explained by the Creative Commons public domain license. +https://creativecommons.org/licenses/publicdomain/ + +This project bundles "org.tukaani:xz:1.5". +This Java implementation of XZ has been put into the public domain, thus you can do +whatever you want with it. All the files in the package have been written by Lasse Collin, +but some files are heavily based on public domain code written by Igor Pavlov. + +This project bundles org.apache.hadoop:*:2.8.1 from which it inherits the following notices: + +The Apache Hadoop project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + +For the org.apache.hadoop.util.bloom.* classes: + +/** + * + * Copyright (c) 2005, European Commission project OneLab under contract + * 034819 (http://www.one-lab.org) + * All rights reserved. + * Redistribution and use in source and binary forms, with or + * without modification, are permitted provided that the following + * conditions are met: + * - Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * - Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the distribution. + * - Neither the name of the University Catholique de Louvain - UCL + * nor the names of its contributors may be used to endorse or + * promote products derived from this software without specific prior + * written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN + * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +For portions of the native implementation of slicing-by-8 CRC calculation +in src/main/native/src/org/apache/hadoop/util: + +/** + * Copyright 2008,2009,2010 Massachusetts Institute of Technology. + * All rights reserved. Use of this source code is governed by a + * BSD-style license that can be found in the LICENSE file. + */ + +For src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,lz4hc.c}, + +/* + LZ4 - Fast LZ compression algorithm + Header File + Copyright (C) 2011-2014, Yann Collet. + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - LZ4 source repository : http://code.google.com/p/lz4/ + - LZ4 public forum : https://groups.google.com/forum/#!forum/lz4c +*/ + +For hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/util/tree.h +--------------------------------------------------------------------- +Copyright 2002 Niels Provos +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles binaries of leveldbjni +(https://github.com/fusesource/leveldbjni), which is available under the +following license: + +Copyright (c) 2011 FuseSource Corp. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of FuseSource Corp. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/compat/{fstatat|openat|unlinkat}.h: + +Copyright (c) 2012 The FreeBSD Foundation +All rights reserved. + +This software was developed by Pawel Jakub Dawidek under sponsorship from +the FreeBSD Foundation. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHORS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHORS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS +OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + +============= + +The binary distribution of this product bundles binaries of leveldb +(http://code.google.com/p/leveldb/), which is available under the following +license: + +Copyright (c) 2011 The LevelDB Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles binaries of snappy +(http://code.google.com/p/snappy/), which is available under the following +license: + +Copyright 2011, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/ +-------------------------------------------------------------------------------- +Copyright (C) 2008-2016, SpryMedia Ltd. + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2010 Aleksander Williams + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2011-2016 Tim Wood, Iskren Chernev, Moment.js contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2 +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css +And the binary distribution of this product bundles these dependencies under the +following license: +Mockito 1.8.5 +SLF4J 1.7.10 +-------------------------------------------------------------------------------- + +The MIT License (MIT) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery +-------------------------------------------------------------------------------- + +Copyright jQuery Foundation and other contributors, https://jquery.org/ + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/jquery/jquery + +The following license applies to all parts of this software except as +documented below: + +==== + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +==== + +All files located in the node_modules and external directories are +externally maintained libraries used by this software which have their +own licenses; we recommend you read them, as their terms may differ from +the terms above. + +For: +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz +-------------------------------------------------------------------------------- + +Copyright (c) 2014 Ivan Bozhanov + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js +-------------------------------------------------------------------------------- + +D3 is available under a 3-clause BSD license. For details, see: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE + +The binary distribution of this product bundles these dependencies under the +following license: +HSQLDB Database 2.0.0 +-------------------------------------------------------------------------------- +"COPYRIGHTS AND LICENSES (based on BSD License) + +For work developed by the HSQL Development Group: + +Copyright (c) 2001-2016, The HSQL Development Group +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +Neither the name of the HSQL Development Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +For work originally developed by the Hypersonic SQL Group: + +Copyright (c) 1995-2000 by the Hypersonic SQL Group. +All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +Neither the name of the Hypersonic SQL Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +This software consists of voluntary contributions made by many individuals on behalf of the +Hypersonic SQL Group." + +The binary distribution of this product bundles these dependencies under the +following license: +servlet-api 2.5 +jsp-api 2.1 +Streaming API for XML 1.0 +-------------------------------------------------------------------------------- +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 +1. Definitions.  + +1.1. Contributor means each individual or entity +that creates or contributes to the creation of +Modifications.  + +1.2. Contributor Version means the combination of the +Original Software, prior Modifications used by a Contributor (if any), and the +Modifications made by that particular Contributor.  + +1.3. Covered +Software means (a) the Original Software, or (b) Modifications, or (c) the +combination of files containing Original Software with files containing +Modifications, in each case including portions +thereof.  + +1.4. Executable means the Covered Software in any form other +than Source Code.  + +1.5. Initial Developer means the individual or entity +that first makes Original Software available under this +License.  + +1.6. Larger Work means a work which combines Covered Software or +portions thereof with code not governed by the terms of this +License.  + +1.7. License means this document.  + +1.8. Licensable means +having the right to grant, to the maximum extent possible, whether at the time +of the initial grant or subsequently acquired, any and all of the rights +conveyed herein.  + +1.9. Modifications means the Source Code and Executable +form of any of the following: +A. Any file that results from an addition to, +deletion from or modification of the contents of a file containing Original +Software or previous Modifications; +B. Any new file that contains any part of the Original Software +or previous Modification; or +C. Any new file that is contributed or otherwise made available +under the terms of this License.  + +1.10. Original Software means the Source Code and Executable form of +computer software code that is originally released under this License.  + +1.11. Patent Claims means any patent claim(s), now owned or +hereafter acquired, including without limitation, method, process, and apparatus +claims, in any patent Licensable by grantor.  + +1.12. Source Code means (a) the common form of computer software code in which +modifications are made and (b) associated documentation included in or +with such code.  + +1.13. You (or Your) means an individual or a legal entity exercising rights +under, and complying with all of the terms of, this License. For legal entities, +You includes any entity which controls, is controlled by, or is under common control +with You. For purposes of this definition, control means (a) the power, direct +or indirect, to cause the direction or management of such entity, whether by +contract or otherwise, or (b) ownership of more than fifty percent (50%) of the +outstanding shares or beneficial ownership of such entity.  + +2. License Grants. + +2.1. The Initial Developer Grant. Conditioned upon Your compliance +with Section 3.1 below and subject to third party intellectual property claims, +the Initial Developer hereby grants You a world-wide, royalty-free, +non-exclusive license:  + +(a) under intellectual property rights (other than +patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, +display, perform, sublicense and distribute the Original Software (or portions +thereof), with or without Modifications, and/or as part of a Larger Work; +and  + +(b) under Patent Claims infringed by the making, using or selling of +Original Software, to make, have made, use, practice, sell, and offer for sale, +and/or otherwise dispose of the Original Software (or portions +thereof); + +(c) The licenses granted in Sections 2.1(a) and (b) are +effective on the date Initial Developer first distributes or otherwise makes the +Original Software available to a third party under the terms of this +License; + +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or +(ii) the combination of the Original Software with other software or +devices.  + +2.2. Contributor Grant. Conditioned upon Your compliance with +Section 3.1 below and subject to third party intellectual property claims, each +Contributor hereby grants You a world-wide, royalty-free, non-exclusive +license:  + +(a) under intellectual property rights (other than patent or +trademark) Licensable by Contributor to use, reproduce, modify, display, +perform, sublicense and distribute the Modifications created by such Contributor +(or portions thereof), either on an unmodified basis, with other Modifications, +as Covered Software and/or as part of a Larger Work; and  + +(b) under Patent +Claims infringed by the making, using, or selling of Modifications made by that +Contributor either alone and/or in combination with its Contributor Version (or +portions of such combination), to make, use, sell, offer for sale, have made, +and/or otherwise dispose of: (1) Modifications made by that Contributor (or +portions thereof); and (2) the combination of Modifications made by that +Contributor with its Contributor Version (or portions of such +combination).  + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are +effective on the date Contributor first distributes or otherwise makes the +Modifications available to a third party. + +(d) Notwithstanding Section 2.2(b) +above, no patent license is granted: (1) for any code that Contributor has +deleted from the Contributor Version; (2) for infringements caused by: +(i) third party modifications of Contributor Version, or (ii) the combination +of Modifications made by that Contributor with other software (except as part of +the Contributor Version) or other devices; or (3) under Patent Claims infringed +by Covered Software in the absence of Modifications made by that +Contributor.  + +3. Distribution Obligations.  + +3.1. Availability of Source +Code. Any Covered Software that You distribute or otherwise make available in +Executable form must also be made available in Source Code form and that Source +Code form must be distributed only under the terms of this License. You must +include a copy of this License with every copy of the Source Code form of the +Covered Software You distribute or otherwise make available. You must inform +recipients of any such Covered Software in Executable form as to how they can +obtain such Covered Software in Source Code form in a reasonable manner on or +through a medium customarily used for software exchange.  + +3.2. +Modifications. The Modifications that You create or to which You contribute are +governed by the terms of this License. You represent that You believe Your +Modifications are Your original creation(s) and/or You have sufficient rights to +grant the rights conveyed by this License.  + +3.3. Required Notices. You must +include a notice in each of Your Modifications that identifies You as the +Contributor of the Modification. You may not remove or alter any copyright, +patent or trademark notices contained within the Covered Software, or any +notices of licensing or any descriptive text giving attribution to any +Contributor or the Initial Developer.  + +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source Code +form that alters or restricts the applicable version of this License or the +recipients rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. + +3.5. Distribution of Executable Versions. You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipients rights in the Source Code form from the rights set forth in this +License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer.  + +3.6. Larger Works. You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software.  + +4. Versions of the License.  + +4.1. +New Versions. Sun Microsystems, Inc. is the initial license steward and may +publish revised and/or new versions of this License from time to time. Each +version will be given a distinguishing version number. Except as provided in +Section 4.3, no one other than the license steward has the right to modify this +License.  + +4.2. Effect of New Versions. You may always continue to use, +distribute or otherwise make the Covered Software available under the terms of +the version of the License under which You originally received the Covered +Software. If the Initial Developer includes a notice in the Original Software +prohibiting it from being distributed or otherwise made available under any +subsequent version of the License, You must distribute and make the Covered +Software available under the terms of the version of the License under which You +originally received the Covered Software. Otherwise, You may also choose to use, +distribute or otherwise make the Covered Software available under the terms of +any subsequent version of the License published by the license +steward.  + +4.3. Modified Versions. When You are an Initial Developer and You +want to create a new license for Your Original Software, You may create and use +a modified version of this License if You: (a) rename the license and remove +any references to the name of the license steward (except to note that the +license differs from this License); and (b) otherwise make it clear that the +license contains terms which differ from this License.  + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, +WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT +LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, +MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY +COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER +OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR +CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS +LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER +THIS DISCLAIMER.  + +6. TERMINATION.  + +6.1. This License and the rights +granted hereunder will terminate automatically if You fail to comply with terms +herein and fail to cure such breach within 30 days of becoming aware of the +breach. Provisions which, by their nature, must remain in effect beyond the +termination of this License shall survive.  + +6.2. If You assert a patent +infringement claim (excluding declaratory judgment actions) against Initial +Developer or a Contributor (the Initial Developer or Contributor against whom +You assert such claim is referred to as Participant) alleging that the +Participant Software (meaning the Contributor Version where the Participant is a +Contributor or the Original Software where the Participant is the Initial +Developer) directly or indirectly infringes any patent, then any and all rights +granted directly or indirectly to You by such Participant, the Initial Developer +(if the Initial Developer is not the Participant) and all Contributors under +Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from +Participant terminate prospectively and automatically at the expiration of such +60 day notice period, unless if within such 60 day period You withdraw Your +claim with respect to the Participant Software against such Participant either +unilaterally or pursuant to a written agreement with Participant.  + +6.3. In +the event of termination under Sections 6.1 or 6.2 above, all end user licenses +that have been validly granted by You or any distributor hereunder prior to +termination (excluding licenses granted to You by any distributor) shall survive +termination.  + +7. LIMITATION OF LIABILITY. +UNDER NO CIRCUMSTANCES AND UNDER +NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, +SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF +COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY +PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY +CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF +GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER +COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE +POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO +LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO +THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT +ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO +THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.  + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a commercial item, as that term is defined in +48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as +that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer +software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. +1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through +227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software +with only those rights set forth herein. This U.S. Government Rights clause is +in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision +that addresses Government rights in computer software under this +License.  + +9. MISCELLANEOUS. +This License represents the complete agreement +concerning subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdictions conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software.  + +10. RESPONSIBILITY FOR CLAIMS. +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability.  + +The binary distribution of this product bundles these dependencies under the +following license: +Jersey 1.9 +JAXB API bundle for GlassFish V3 2.2.2 +JAXB RI 2.2.3 +-------------------------------------------------------------------------------- +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 + +1. Definitions. + +1.1. “Contributor” means each individual or entity that creates or +contributes to the creation of Modifications. +1.2. “Contributor Version” means the combination of the Original Software, +prior Modifications used by a Contributor (if any), and the Modifications made +by that particular Contributor. +1.3. “Covered Software” means (a) the Original Software, or (b) +Modifications, or (c) the combination of files containing Original Software with +files containing Modifications, in each case including portions thereof. +1.4. “Executable” means the Covered Software in any form other than Source +Code. +1.5. “Initial Developer” means the individual or entity that first makes +Original Software available under this License. +1.6. “Larger Work” means a work which combines Covered Software or portions +thereof with code not governed by the terms of this License. +1.7. “License” means this document. +1.8. “Licensable” means having the right to grant, to the maximum extent +possible, whether at the time of the initial grant or subsequently acquired, any +and all of the rights conveyed herein. +1.9. “Modifications” means the Source Code and Executable form of any of the +following: +A. Any file that results from an addition to, deletion from or modification of +the contents of a file containing Original Software or previous Modifications; +B. Any new file that contains any part of the Original Software or previous +Modification; or +C. Any new file that is contributed or otherwise made available under the terms +of this License. +1.10. “Original Software” means the Source Code and Executable form of +computer software code that is originally released under this License. +1.11. “Patent Claims” means any patent claim(s), now owned or hereafter +acquired, including without limitation, method, process, and apparatus claims, +in any patent Licensable by grantor. +1.12. “Source Code” means (a) the common form of computer software code in +which modifications are made and (b) associated documentation included in or +with such code. +1.13. “You” (or “Your”) means an individual or a legal entity exercising +rights under, and complying with all of the terms of, this License. For legal +entities, “You” includes any entity which controls, is controlled by, or is +under common control with You. For purposes of this definition, “control” +means (a) the power, direct or indirect, to cause the direction or management of +such entity, whether by contract or otherwise, or (b) ownership of more than +fifty percent (50%) of the outstanding shares or beneficial ownership of such +entity. + +2. License Grants. + +2.1. The Initial Developer Grant. + +Conditioned upon Your compliance with Section 3.1 below and subject to +third party intellectual property claims, the Initial Developer hereby grants +You a world-wide, royalty-free, non-exclusive license: +(a) under intellectual +property rights (other than patent or trademark) Licensable by Initial +Developer, to use, reproduce, modify, display, perform, sublicense and +distribute the Original Software (or portions thereof), with or without +Modifications, and/or as part of a Larger Work; and +(b) under Patent Claims +infringed by the making, using or selling of Original Software, to make, have +made, use, practice, sell, and offer for sale, and/or otherwise dispose of the +Original Software (or portions thereof). +(c) The licenses granted in Sections +2.1(a) and (b) are effective on the date Initial Developer first distributes or +otherwise makes the Original Software available to a third party under the terms +of this License. +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or (ii) +the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. + +Conditioned upon Your compliance with Section 3.1 below and +subject to third party intellectual property claims, each Contributor hereby +grants You a world-wide, royalty-free, non-exclusive license: +(a) under +intellectual property rights (other than patent or trademark) Licensable by +Contributor to use, reproduce, modify, display, perform, sublicense and +distribute the Modifications created by such Contributor (or portions thereof), +either on an unmodified basis, with other Modifications, as Covered Software +and/or as part of a Larger Work; and +(b) under Patent Claims infringed by the +making, using, or selling of Modifications made by that Contributor either alone +and/or in combination with its Contributor Version (or portions of such +combination), to make, use, sell, offer for sale, have made, and/or otherwise +dispose of: (1) Modifications made by that Contributor (or portions thereof); +and (2) the combination of Modifications made by that Contributor with its +Contributor Version (or portions of such combination). +(c) The licenses granted +in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first +distributes or otherwise makes the Modifications available to a third +party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: +(1) for any code that Contributor has deleted from the Contributor Version; (2) +for infringements caused by: (i) third party modifications of Contributor +Version, or (ii) the combination of Modifications made by that Contributor with +other software (except as part of the Contributor Version) or other devices; or +(3) under Patent Claims infringed by Covered Software in the absence of +Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. +Any Covered Software that You distribute or +otherwise make available in Executable form must also be made available in +Source Code form and that Source Code form must be distributed only under the +terms of this License. You must include a copy of this License with every copy +of the Source Code form of the Covered Software You distribute or otherwise make +available. You must inform recipients of any such Covered Software in Executable +form as to how they can obtain such Covered Software in Source Code form in a +reasonable manner on or through a medium customarily used for software +exchange. +3.2. Modifications. +The Modifications that You create or to which +You contribute are governed by the terms of this License. You represent that You +believe Your Modifications are Your original creation(s) and/or You have +sufficient rights to grant the rights conveyed by this License. +3.3. Required Notices. +You must include a notice in each of Your Modifications that +identifies You as the Contributor of the Modification. You may not remove or +alter any copyright, patent or trademark notices contained within the Covered +Software, or any notices of licensing or any descriptive text giving attribution +to any Contributor or the Initial Developer. +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source +Code form that alters or restricts the applicable version of this License or the +recipients' rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. +3.5. Distribution of Executable Versions. +You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipient's rights in the Source Code form from the rights set forth in +this License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer. +3.6. Larger Works. +You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. +Oracle is the initial license steward and may publish revised and/or +new versions of this License from time to time. Each version will be given a +distinguishing version number. Except as provided in Section 4.3, no one other +than the license steward has the right to modify this License. +4.2. Effect of New Versions. +You may always continue to use, distribute or otherwise make the +Covered Software available under the terms of the version of the License under +which You originally received the Covered Software. If the Initial Developer +includes a notice in the Original Software prohibiting it from being distributed +or otherwise made available under any subsequent version of the License, You +must distribute and make the Covered Software available under the terms of the +version of the License under which You originally received the Covered Software. +Otherwise, You may also choose to use, distribute or otherwise make the Covered +Software available under the terms of any subsequent version of the License +published by the license steward. +4.3. Modified Versions. +When You are an +Initial Developer and You want to create a new license for Your Original +Software, You may create and use a modified version of this License if You: (a) +rename the license and remove any references to the name of the license steward +(except to note that the license differs from this License); and (b) otherwise +make it clear that the license contains terms which differ from this +License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS +LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE +IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR +NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED +SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY +RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE +COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF +WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED +SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will +terminate automatically if You fail to comply with terms herein and fail to cure +such breach within 30 days of becoming aware of the breach. Provisions which, by +their nature, must remain in effect beyond the termination of this License shall +survive. +6.2. If You assert a patent infringement claim (excluding declaratory +judgment actions) against Initial Developer or a Contributor (the Initial +Developer or Contributor against whom You assert such claim is referred to as +“Participant”) alleging that the Participant Software (meaning the +Contributor Version where the Participant is a Contributor or the Original +Software where the Participant is the Initial Developer) directly or indirectly +infringes any patent, then any and all rights granted directly or indirectly to +You by such Participant, the Initial Developer (if the Initial Developer is not +the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this +License shall, upon 60 days notice from Participant terminate prospectively and +automatically at the expiration of such 60 day notice period, unless if within +such 60 day period You withdraw Your claim with respect to the Participant +Software against such Participant either unilaterally or pursuant to a written +agreement with Participant. +6.3. If You assert a patent infringement claim +against Participant alleging that the Participant Software directly or +indirectly infringes any patent where such claim is resolved (such as by license +or settlement) prior to the initiation of patent infringement litigation, then +the reasonable value of the licenses granted by such Participant under Sections +2.1 or 2.2 shall be taken into account in determining the amount or value of any +payment or license. +6.4. In the event of termination under Sections 6.1 or 6.2 +above, all end user licenses that have been validly granted by You or any +distributor hereunder prior to termination (excluding licenses granted to You by +any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT +(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL +DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY +SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, +SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, +WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER +FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN +IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS +LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL +INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW +PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR +LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND +LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + +The Covered +Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 +(Oct. 1995), consisting of “commercial computer software” (as that term is +defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software +documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). +Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 +(June 1995), all U.S. Government End Users acquire Covered Software with only +those rights set forth herein. This U.S. Government Rights clause is in lieu of, +and supersedes, any other FAR, DFAR, or other clause or provision that addresses +Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning +subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdiction's conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys' fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability. + +The binary distribution of this product bundles these dependencies under the +following license: +Protocol Buffer Java API 2.5.0 +-------------------------------------------------------------------------------- +This license applies to all parts of Protocol Buffers except the following: + + - Atomicops support for generic gcc, located in + src/google/protobuf/stubs/atomicops_internals_generic_gcc.h. + This file is copyrighted by Red Hat Inc. + + - Atomicops support for AIX/POWER, located in + src/google/protobuf/stubs/atomicops_internals_power.h. + This file is copyrighted by Bloomberg Finance LP. + +Copyright 2014, Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +For: +XML Commons External Components XML APIs 1.3.04 +-------------------------------------------------------------------------------- +By obtaining, using and/or copying this work, you (the licensee) agree that you +have read, understood, and will comply with the following terms and conditions. + +Permission to copy, modify, and distribute this software and its documentation, +with or without modification, for any purpose and without fee or royalty is +hereby granted, provided that you include the following on ALL copies of the +software and documentation or portions thereof, including modifications: +- The full text of this NOTICE in a location viewable to users of the +redistributed or derivative work. +- Any pre-existing intellectual property disclaimers, notices, or terms and +conditions. If none exist, the W3C Software Short Notice should be included +(hypertext is preferred, text is permitted) within the body of any redistributed +or derivative code. +- Notice of any changes or modifications to the files, including the date changes +were made. (We recommend you provide URIs to the location from which the code is +derived.) + +The binary distribution of this product bundles these dependencies under the +following license: +JUnit 4.11 +ecj-4.3.1.jar +-------------------------------------------------------------------------------- +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC +LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM +CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation +distributed under this Agreement, and +b) in the case of each subsequent Contributor: +i) changes to the Program, and +ii) additions to the Program; +where such changes and/or additions to the Program originate from and are +distributed by that particular Contributor. A Contribution 'originates' from a +Contributor if it was added to the Program by such Contributor itself or anyone +acting on such Contributor's behalf. Contributions do not include additions to +the Program which: (i) are separate modules of software distributed in +conjunction with the Program under their own license agreement, and (ii) are not +derivative works of the Program. +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are +necessarily infringed by the use or sale of its Contribution alone or when +combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, +including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free copyright license to +reproduce, prepare derivative works of, publicly display, publicly perform, +distribute and sublicense the Contribution of such Contributor, if any, and such +derivative works, in source code and object code form. +b) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed +Patents to make, use, sell, offer to sell, import and otherwise transfer the +Contribution of such Contributor, if any, in source code and object code form. +This patent license shall apply to the combination of the Contribution and the +Program if, at the time the Contribution is added by the Contributor, such +addition of the Contribution causes such combination to be covered by the +Licensed Patents. The patent license shall not apply to any other combinations +which include the Contribution. No hardware per se is licensed hereunder. +c) Recipient understands that although each Contributor grants the licenses to +its Contributions set forth herein, no assurances are provided by any +Contributor that the Program does not infringe the patent or other intellectual +property rights of any other entity. Each Contributor disclaims any liability to +Recipient for claims brought by any other entity based on infringement of +intellectual property rights or otherwise. As a condition to exercising the +rights and licenses granted hereunder, each Recipient hereby assumes sole +responsibility to secure any other intellectual property rights needed, if any. +For example, if a third party patent license is required to allow Recipient to +distribute the Program, it is Recipient's responsibility to acquire that license +before distributing the Program. +d) Each Contributor represents that to its knowledge it has sufficient copyright +rights in its Contribution, if any, to grant the copyright license set forth in +this Agreement. +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its +own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and +b) its license agreement: +i) effectively disclaims on behalf of all Contributors all warranties and +conditions, express and implied, including warranties or conditions of title and +non-infringement, and implied warranties or conditions of merchantability and +fitness for a particular purpose; +ii) effectively excludes on behalf of all Contributors all liability for +damages, including direct, indirect, special, incidental and consequential +damages, such as lost profits; +iii) states that any provisions which differ from this Agreement are offered by +that Contributor alone and not by any other party; and +iv) states that source code for the Program is available from such Contributor, +and informs licensees how to obtain it in a reasonable manner on or through a +medium customarily used for software exchange. +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and +b) a copy of this Agreement must be included with each copy of the Program. +Contributors may not remove or alter any copyright notices contained within the +Program. + +Each Contributor must identify itself as the originator of its Contribution, if +any, in a manner that reasonably allows subsequent Recipients to identify the +originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with +respect to end users, business partners and the like. While this license is +intended to facilitate the commercial use of the Program, the Contributor who +includes the Program in a commercial product offering should do so in a manner +which does not create potential liability for other Contributors. Therefore, if +a Contributor includes the Program in a commercial product offering, such +Contributor ("Commercial Contributor") hereby agrees to defend and indemnify +every other Contributor ("Indemnified Contributor") against any losses, damages +and costs (collectively "Losses") arising from claims, lawsuits and other legal +actions brought by a third party against the Indemnified Contributor to the +extent caused by the acts or omissions of such Commercial Contributor in +connection with its distribution of the Program in a commercial product +offering. The obligations in this section do not apply to any claims or Losses +relating to any actual or alleged intellectual property infringement. In order +to qualify, an Indemnified Contributor must: a) promptly notify the Commercial +Contributor in writing of such claim, and b) allow the Commercial Contributor to +control, and cooperate with the Commercial Contributor in, the defense and any +related settlement negotiations. The Indemnified Contributor may participate in +any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product +offering, Product X. That Contributor is then a Commercial Contributor. If that +Commercial Contributor then makes performance claims, or offers warranties +related to Product X, those performance claims and warranties are such +Commercial Contributor's responsibility alone. Under this section, the +Commercial Contributor would have to defend claims against the other +Contributors related to those performance claims and warranties, and if a court +requires any other Contributor to pay any damages as a result, the Commercial +Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR +IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, +NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each +Recipient is solely responsible for determining the appropriateness of using and +distributing the Program and assumes all risks associated with its exercise of +rights under this Agreement , including but not limited to the risks and costs +of program errors, compliance with applicable laws, damage to or loss of data, +programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY +CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST +PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS +GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this Agreement, and without further action by the parties hereto, such +provision shall be reformed to the minimum extent necessary to make such +provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a +cross-claim or counterclaim in a lawsuit) alleging that the Program itself +(excluding combinations of the Program with other software or hardware) +infringes such Recipient's patent(s), then such Recipient's rights granted under +Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to +comply with any of the material terms or conditions of this Agreement and does +not cure such failure in a reasonable period of time after becoming aware of +such noncompliance. If all Recipient's rights under this Agreement terminate, +Recipient agrees to cease use and distribution of the Program as soon as +reasonably practicable. However, Recipient's obligations under this Agreement +and any licenses granted by Recipient relating to the Program shall continue and +survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in +order to avoid inconsistency the Agreement is copyrighted and may only be +modified in the following manner. The Agreement Steward reserves the right to +publish new versions (including revisions) of this Agreement from time to time. +No one other than the Agreement Steward has the right to modify this Agreement. +The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation +may assign the responsibility to serve as the Agreement Steward to a suitable +separate entity. Each new version of the Agreement will be given a +distinguishing version number. The Program (including Contributions) may always +be distributed subject to the version of the Agreement under which it was +received. In addition, after a new version of the Agreement is published, +Contributor may elect to distribute the Program (including its Contributions) +under the new version. Except as expressly stated in Sections 2(a) and 2(b) +above, Recipient receives no rights or licenses to the intellectual property of +any Contributor under this Agreement, whether expressly, by implication, +estoppel or otherwise. All rights in the Program not expressly granted under +this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the +intellectual property laws of the United States of America. No party to this +Agreement will bring a legal action under this Agreement more than one year +after the cause of action arose. Each party waives its rights to a jury trial in +any resulting litigation. + +The binary distribution of this product bundles these dependencies under the +following license: +ASM Core 3.2 +JSch 0.1.51 +ParaNamer Core 2.3 +JLine 0.9.94 +leveldbjni-all 1.8 +Hamcrest Core 1.3 +xmlenc Library 0.52 +-------------------------------------------------------------------------------- +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles these dependencies under the +following license: +FindBugs-jsr305 3.0.0 +-------------------------------------------------------------------------------- +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The views and conclusions contained in the software and documentation are those +of the authors and should not be interpreted as representing official policies, +either expressed or implied, of the FreeBSD Project. + +The binary distribution of this product bundles these dependencies under the +following license: +Java Concurrency in Practice book annotations 1.0 +-------------------------------------------------------------------------------- +THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE COMMONS +PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY COPYRIGHT AND/OR +OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS AUTHORIZED UNDER THIS +LICENSE OR COPYRIGHT LAW IS PROHIBITED. + +BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE TO BE +BOUND BY THE TERMS OF THIS LICENSE. THE LICENSOR GRANTS YOU THE RIGHTS CONTAINED +HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND CONDITIONS. + +1. Definitions + +"Collective Work" means a work, such as a periodical issue, anthology or +encyclopedia, in which the Work in its entirety in unmodified form, along with a +number of other contributions, constituting separate and independent works in +themselves, are assembled into a collective whole. A work that constitutes a +Collective Work will not be considered a Derivative Work (as defined below) for +the purposes of this License. +"Derivative Work" means a work based upon the Work or upon the Work and other +pre-existing works, such as a translation, musical arrangement, dramatization, +fictionalization, motion picture version, sound recording, art reproduction, +abridgment, condensation, or any other form in which the Work may be recast, +transformed, or adapted, except that a work that constitutes a Collective Work +will not be considered a Derivative Work for the purpose of this License. For +the avoidance of doubt, where the Work is a musical composition or sound +recording, the synchronization of the Work in timed-relation with a moving image +("synching") will be considered a Derivative Work for the purpose of this +License. +"Licensor" means the individual or entity that offers the Work under the terms +of this License. +"Original Author" means the individual or entity who created the Work. +"Work" means the copyrightable work of authorship offered under the terms of +this License. +"You" means an individual or entity exercising rights under this License who has +not previously violated the terms of this License with respect to the Work, or +who has received express permission from the Licensor to exercise rights under +this License despite a previous violation. +2. Fair Use Rights. Nothing in this license is intended to reduce, limit, or +restrict any rights arising from fair use, first sale or other limitations on +the exclusive rights of the copyright owner under copyright law or other +applicable laws. + +3. License Grant. Subject to the terms and conditions of this License, Licensor +hereby grants You a worldwide, royalty-free, non-exclusive, perpetual (for the +duration of the applicable copyright) license to exercise the rights in the Work +as stated below: + +to reproduce the Work, to incorporate the Work into one or more Collective +Works, and to reproduce the Work as incorporated in the Collective Works; +to create and reproduce Derivative Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission the Work including as +incorporated in Collective Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission Derivative Works. +For the avoidance of doubt, where the work is a musical composition: + +Performance Royalties Under Blanket Licenses. Licensor waives the exclusive +right to collect, whether individually or via a performance rights society (e.g. +ASCAP, BMI, SESAC), royalties for the public performance or public digital +performance (e.g. webcast) of the Work. +Mechanical Rights and Statutory Royalties. Licensor waives the exclusive right +to collect, whether individually or via a music rights agency or designated +agent (e.g. Harry Fox Agency), royalties for any phonorecord You create from the +Work ("cover version") and distribute, subject to the compulsory license created +by 17 USC Section 115 of the US Copyright Act (or the equivalent in other +jurisdictions). +Webcasting Rights and Statutory Royalties. For the avoidance of doubt, where the +Work is a sound recording, Licensor waives the exclusive right to collect, +whether individually or via a performance-rights society (e.g. SoundExchange), +royalties for the public digital performance (e.g. webcast) of the Work, subject +to the compulsory license created by 17 USC Section 114 of the US Copyright Act +(or the equivalent in other jurisdictions). +The above rights may be exercised in all media and formats whether now known or +hereafter devised. The above rights include the right to make such modifications +as are technically necessary to exercise the rights in other media and formats. +All rights not expressly granted by Licensor are hereby reserved. + +4. Restrictions.The license granted in Section 3 above is expressly made subject +to and limited by the following restrictions: + +You may distribute, publicly display, publicly perform, or publicly digitally +perform the Work only under the terms of this License, and You must include a +copy of, or the Uniform Resource Identifier for, this License with every copy or +phonorecord of the Work You distribute, publicly display, publicly perform, or +publicly digitally perform. You may not offer or impose any terms on the Work +that alter or restrict the terms of this License or the recipients' exercise of +the rights granted hereunder. You may not sublicense the Work. You must keep +intact all notices that refer to this License and to the disclaimer of +warranties. You may not distribute, publicly display, publicly perform, or +publicly digitally perform the Work with any technological measures that control +access or use of the Work in a manner inconsistent with the terms of this +License Agreement. The above applies to the Work as incorporated in a Collective +Work, but this does not require the Collective Work apart from the Work itself +to be made subject to the terms of this License. If You create a Collective +Work, upon notice from any Licensor You must, to the extent practicable, remove +from the Collective Work any credit as required by clause 4(b), as requested. If +You create a Derivative Work, upon notice from any Licensor You must, to the +extent practicable, remove from the Derivative Work any credit as required by +clause 4(b), as requested. +If you distribute, publicly display, publicly perform, or publicly digitally +perform the Work or any Derivative Works or Collective Works, You must keep +intact all copyright notices for the Work and provide, reasonable to the medium +or means You are utilizing: (i) the name of the Original Author (or pseudonym, +if applicable) if supplied, and/or (ii) if the Original Author and/or Licensor +designate another party or parties (e.g. a sponsor institute, publishing entity, +journal) for attribution in Licensor's copyright notice, terms of service or by +other reasonable means, the name of such party or parties; the title of the Work +if supplied; to the extent reasonably practicable, the Uniform Resource +Identifier, if any, that Licensor specifies to be associated with the Work, +unless such URI does not refer to the copyright notice or licensing information +for the Work; and in the case of a Derivative Work, a credit identifying the use +of the Work in the Derivative Work (e.g., "French translation of the Work by +Original Author," or "Screenplay based on original Work by Original Author"). +Such credit may be implemented in any reasonable manner; provided, however, that +in the case of a Derivative Work or Collective Work, at a minimum such credit +will appear where any other comparable authorship credit appears and in a manner +at least as prominent as such other comparable authorship credit. +5. Representations, Warranties and Disclaimer + +UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR OFFERS +THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY KIND CONCERNING +THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE, INCLUDING, WITHOUT +LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY, FITNESS FOR A PARTICULAR +PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF LATENT OR OTHER DEFECTS, ACCURACY, +OR THE PRESENCE OF ABSENCE OF ERRORS, WHETHER OR NOT DISCOVERABLE. SOME +JURISDICTIONS DO NOT ALLOW THE EXCLUSION OF IMPLIED WARRANTIES, SO SUCH +EXCLUSION MAY NOT APPLY TO YOU. + +6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE LAW, IN +NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR ANY SPECIAL, +INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES ARISING OUT OF THIS +LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + +7. Termination + +This License and the rights granted hereunder will terminate automatically upon +any breach by You of the terms of this License. Individuals or entities who have +received Derivative Works or Collective Works from You under this License, +however, will not have their licenses terminated provided such individuals or +entities remain in full compliance with those licenses. Sections 1, 2, 5, 6, 7, +and 8 will survive any termination of this License. +Subject to the above terms and conditions, the license granted here is perpetual +(for the duration of the applicable copyright in the Work). Notwithstanding the +above, Licensor reserves the right to release the Work under different license +terms or to stop distributing the Work at any time; provided, however that any +such election will not serve to withdraw this License (or any other license that +has been, or is required to be, granted under the terms of this License), and +this License will continue in full force and effect unless terminated as stated +above. +8. Miscellaneous + +Each time You distribute or publicly digitally perform the Work or a Collective +Work, the Licensor offers to the recipient a license to the Work on the same +terms and conditions as the license granted to You under this License. +Each time You distribute or publicly digitally perform a Derivative Work, +Licensor offers to the recipient a license to the original Work on the same +terms and conditions as the license granted to You under this License. +If any provision of this License is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this License, and without further action by the parties to this +agreement, such provision shall be reformed to the minimum extent necessary to +make such provision valid and enforceable. +No term or provision of this License shall be deemed waived and no breach +consented to unless such waiver or consent shall be in writing and signed by the +party to be charged with such waiver or consent. +This License constitutes the entire agreement between the parties with respect +to the Work licensed here. There are no understandings, agreements or +representations with respect to the Work not specified here. Licensor shall not +be bound by any additional provisions that may appear in any communication from +You. This License may not be modified without the mutual written agreement of +the Licensor and You. + diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-protobuf b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-protobuf new file mode 100644 index 0000000000000..21645bec9d5d1 --- /dev/null +++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-protobuf @@ -0,0 +1,36 @@ +Copyright 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-xmlenc b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-xmlenc new file mode 100644 index 0000000000000..a1256bdf6f8fd --- /dev/null +++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/licenses/LICENSE-xmlenc @@ -0,0 +1,28 @@ +Copyright 2003-2005, Ernst de Haan +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + From 334b12b3535c38c816f7c51113eb7a956f4f4222 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 25 Nov 2018 12:35:52 +0100 Subject: [PATCH 165/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-dist --- flink-dist/pom.xml | 4 +- flink-dist/src/main/resources/META-INF/NOTICE | 60 +++++++++++++++++++ .../META-INF/licenses/LICENSE.grizzled-slf4j | 11 ++++ .../resources/META-INF/licenses/LICENSE.kryo | 10 ++++ .../META-INF/licenses/LICENSE.minlog | 10 ++++ .../resources/META-INF/licenses/LICENSE.scala | 11 ++++ .../resources/META-INF/licenses/LICENSE.scopt | 21 +++++++ .../META-INF/licenses/LICENSE.slf4j-api | 21 +++++++ 8 files changed, 147 insertions(+), 1 deletion(-) create mode 100644 flink-dist/src/main/resources/META-INF/NOTICE create mode 100644 flink-dist/src/main/resources/META-INF/licenses/LICENSE.grizzled-slf4j create mode 100644 flink-dist/src/main/resources/META-INF/licenses/LICENSE.kryo create mode 100644 flink-dist/src/main/resources/META-INF/licenses/LICENSE.minlog create mode 100644 flink-dist/src/main/resources/META-INF/licenses/LICENSE.scala create mode 100644 flink-dist/src/main/resources/META-INF/licenses/LICENSE.scopt create mode 100644 flink-dist/src/main/resources/META-INF/licenses/LICENSE.slf4j-api diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index d58f6f801479f..fc0bb02d2077a 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -542,7 +542,9 @@ under the License. - + + Apache Flink + diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..7088b1ab775bc --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/NOTICE @@ -0,0 +1,60 @@ +flink-dist +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.code.findbugs:jsr305:1.3.9 +- com.twitter:chill-java:0.7.6 +- com.twitter:chill_2.11:0.7.6 +- com.typesafe:config:1.3.0 +- com.typesafe:ssl-config-core_2.11:0.2.1 +- com.typesafe.akka:akka-actor_2.11:2.4.20 +- com.typesafe.akka:akka-camel_2.11:2.4.20 +- com.typesafe.akka:akka-protobuf_2.11:2.4.20 +- com.typesafe.akka:akka-slf4j_2.11:2.4.20 +- com.typesafe.akka:akka-stream_2.11:2.4.20 +- commons-cli:commons-cli:1.3.1 +- commons-collections:commons-collections:3.2.2 +- commons-io:commons-io:2.4 +- org.apache.camel:camel-core:2.17.7 +- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-lang3:3.3.2 +- org.apache.commons:commons-math3:3.5 +- org.javassist:javassist:3.19.0-GA +- org.objenesis:objenesis:2.1 +- org.rocksdb:rocksdbjni:5.7.5 +- org.xerial.snappy:snappy-java:1.1.4 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.esotericsoftware.kryo:kryo:2.24.0 +- com.esotericsoftware.minlog:minlog:1.2 +- org.clapper:grizzled-slf4j_2.11:1.3.2 + +The following dependencies all share the same BSD license which you find under licenses/LICENSE.scala. + +- org.scala-lang:scala-compiler:2.11.12 +- org.scala-lang:scala-library:2.11.12 +- org.scala-lang:scala-reflect:2.11.12 +- org.scala-lang.modules:scala-java8-compat_2.11:0.7.0 +- org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4 +- org.scala-lang.modules:scala-xml_2.11:1.0.5 + +This project bundles the following dependencies under the MIT/X11 license. +See bundled license files for details. + +- com.github.scopt:scopt_2.11:3.5.0 +- org.slf4j:slf4j-api:1.7.15 + +This project bundles "org.tukaani:xz:1.5". +This Java implementation of XZ has been put into the public domain, thus you can do +whatever you want with it. All the files in the package have been written by Lasse Collin, +but some files are heavily based on public domain code written by Igor Pavlov. + +This project bundles the following dependencies under the Creative Commons CC0 "No Rights Reserved". + +- org.reactivestreams:reactive-streams:1.0.0 diff --git a/flink-dist/src/main/resources/META-INF/licenses/LICENSE.grizzled-slf4j b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.grizzled-slf4j new file mode 100644 index 0000000000000..6601c0131eeac --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.grizzled-slf4j @@ -0,0 +1,11 @@ +Copyright © 2010-2012 Brian M. Clapper. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + +Neither the name “clapper.org” nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-dist/src/main/resources/META-INF/licenses/LICENSE.kryo b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.kryo new file mode 100644 index 0000000000000..e1cd88478edf7 --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.kryo @@ -0,0 +1,10 @@ +Copyright (c) 2008, Nathan Sweet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-dist/src/main/resources/META-INF/licenses/LICENSE.minlog b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.minlog new file mode 100644 index 0000000000000..e1cd88478edf7 --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.minlog @@ -0,0 +1,10 @@ +Copyright (c) 2008, Nathan Sweet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-dist/src/main/resources/META-INF/licenses/LICENSE.scala b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.scala new file mode 100644 index 0000000000000..6d8bdabbbe0fb --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.scala @@ -0,0 +1,11 @@ +Copyright (c) 2002- EPFL +Copyright (c) 2011- Lightbend, Inc. + +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. +Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. +Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-dist/src/main/resources/META-INF/licenses/LICENSE.scopt b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.scopt new file mode 100644 index 0000000000000..6d6a875878f4b --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.scopt @@ -0,0 +1,21 @@ +This project is licensed under the [MIT license](https://en.wikipedia.org/wiki/MIT_License). + +Copyright (c) scopt contributors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE +OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/flink-dist/src/main/resources/META-INF/licenses/LICENSE.slf4j-api b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.slf4j-api new file mode 100644 index 0000000000000..93119e70ed5e7 --- /dev/null +++ b/flink-dist/src/main/resources/META-INF/licenses/LICENSE.slf4j-api @@ -0,0 +1,21 @@ +Copyright (c) 2004-2017 QOS.ch + All rights reserved. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. From 95f6531a86bec62266f6c9588d2f1a2716555def Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Nov 2018 14:48:19 +0100 Subject: [PATCH 166/359] [FLINK-10987] Add NOTICE & licenses for binary release Adds a NOTICE-binary file and a licenses-binary directory containing all bundled licenses for the binary release. When creating the binary release the NOTICE-binary file and the licenses-binary directory will be copied and renamed to NOTICE and the licenses in the binary release. --- NOTICE-binary | 5808 +++++++++++++++++ flink-dist/src/main/assemblies/bin.xml | 29 +- licenses-binary/LICENSE-hdrhistogram | 125 + licenses-binary/LICENSE-protobuf | 36 + licenses-binary/LICENSE-re2j | 32 + licenses-binary/LICENSE-stax2api | 22 + licenses-binary/LICENSE-xmlenc | 28 + licenses-binary/LICENSE.angular | 22 + .../LICENSE.angular-drag-and-drop-list | 22 + licenses-binary/LICENSE.angular-moment | 21 + licenses-binary/LICENSE.angular-ui-router | 21 + licenses-binary/LICENSE.arpack_combined_all | 8 + licenses-binary/LICENSE.asm.txt | 31 + licenses-binary/LICENSE.base64 | 26 + licenses-binary/LICENSE.bootstrap | 21 + licenses-binary/LICENSE.cddlv1.0 | 129 + licenses-binary/LICENSE.cddlv1.1 | 348 + licenses-binary/LICENSE.core | 49 + licenses-binary/LICENSE.d3 | 27 + licenses-binary/LICENSE.dagre | 19 + licenses-binary/LICENSE.dagre-d3 | 19 + licenses-binary/LICENSE.ev-emitter | 7 + licenses-binary/LICENSE.font-awesome | 97 + licenses-binary/LICENSE.graphlib | 19 + licenses-binary/LICENSE.grizzled-slf4j | 11 + licenses-binary/LICENSE.imagesloaded | 7 + licenses-binary/LICENSE.janino | 31 + licenses-binary/LICENSE.jline | 7 + licenses-binary/LICENSE.jquery | 20 + licenses-binary/LICENSE.jsch | 30 + licenses-binary/LICENSE.jsr166y | 26 + licenses-binary/LICENSE.jtransforms | 480 ++ licenses-binary/LICENSE.jython | 17 + licenses-binary/LICENSE.jzlib | 26 + licenses-binary/LICENSE.kryo | 10 + licenses-binary/LICENSE.lodash | 10 + licenses-binary/LICENSE.machinist | 19 + licenses-binary/LICENSE.minlog | 10 + licenses-binary/LICENSE.moment | 22 + .../LICENSE.moment-duration-format | 21 + licenses-binary/LICENSE.paranamer | 28 + licenses-binary/LICENSE.protobuf | 32 + licenses-binary/LICENSE.qtip2 | 22 + licenses-binary/LICENSE.scala | 11 + licenses-binary/LICENSE.scopt | 21 + licenses-binary/LICENSE.slf4j | 21 + licenses-binary/LICENSE.slf4j-api | 21 + licenses-binary/LICENSE.spire | 19 + licenses-binary/LICENSE.split | 32 + licenses-binary/LICENSE.webbit | 38 + licenses-binary/LICENSE.xmlenc | 27 + pom.xml | 1 + tools/releasing/collect_license_files.sh | 57 + 53 files changed, 8032 insertions(+), 11 deletions(-) create mode 100644 NOTICE-binary create mode 100644 licenses-binary/LICENSE-hdrhistogram create mode 100644 licenses-binary/LICENSE-protobuf create mode 100644 licenses-binary/LICENSE-re2j create mode 100644 licenses-binary/LICENSE-stax2api create mode 100644 licenses-binary/LICENSE-xmlenc create mode 100644 licenses-binary/LICENSE.angular create mode 100644 licenses-binary/LICENSE.angular-drag-and-drop-list create mode 100644 licenses-binary/LICENSE.angular-moment create mode 100644 licenses-binary/LICENSE.angular-ui-router create mode 100644 licenses-binary/LICENSE.arpack_combined_all create mode 100644 licenses-binary/LICENSE.asm.txt create mode 100644 licenses-binary/LICENSE.base64 create mode 100644 licenses-binary/LICENSE.bootstrap create mode 100644 licenses-binary/LICENSE.cddlv1.0 create mode 100644 licenses-binary/LICENSE.cddlv1.1 create mode 100644 licenses-binary/LICENSE.core create mode 100644 licenses-binary/LICENSE.d3 create mode 100644 licenses-binary/LICENSE.dagre create mode 100644 licenses-binary/LICENSE.dagre-d3 create mode 100644 licenses-binary/LICENSE.ev-emitter create mode 100644 licenses-binary/LICENSE.font-awesome create mode 100644 licenses-binary/LICENSE.graphlib create mode 100644 licenses-binary/LICENSE.grizzled-slf4j create mode 100644 licenses-binary/LICENSE.imagesloaded create mode 100644 licenses-binary/LICENSE.janino create mode 100644 licenses-binary/LICENSE.jline create mode 100644 licenses-binary/LICENSE.jquery create mode 100644 licenses-binary/LICENSE.jsch create mode 100644 licenses-binary/LICENSE.jsr166y create mode 100644 licenses-binary/LICENSE.jtransforms create mode 100644 licenses-binary/LICENSE.jython create mode 100644 licenses-binary/LICENSE.jzlib create mode 100644 licenses-binary/LICENSE.kryo create mode 100644 licenses-binary/LICENSE.lodash create mode 100644 licenses-binary/LICENSE.machinist create mode 100644 licenses-binary/LICENSE.minlog create mode 100644 licenses-binary/LICENSE.moment create mode 100644 licenses-binary/LICENSE.moment-duration-format create mode 100644 licenses-binary/LICENSE.paranamer create mode 100644 licenses-binary/LICENSE.protobuf create mode 100644 licenses-binary/LICENSE.qtip2 create mode 100644 licenses-binary/LICENSE.scala create mode 100644 licenses-binary/LICENSE.scopt create mode 100644 licenses-binary/LICENSE.slf4j create mode 100644 licenses-binary/LICENSE.slf4j-api create mode 100644 licenses-binary/LICENSE.spire create mode 100644 licenses-binary/LICENSE.split create mode 100644 licenses-binary/LICENSE.webbit create mode 100644 licenses-binary/LICENSE.xmlenc create mode 100755 tools/releasing/collect_license_files.sh diff --git a/NOTICE-binary b/NOTICE-binary new file mode 100644 index 0000000000000..74df3d1c03ddd --- /dev/null +++ b/NOTICE-binary @@ -0,0 +1,5808 @@ +// ------------------------------------------------------------------ +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for Apache Flink +// ------------------------------------------------------------------ + +Apache Flink +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.chuusai:shapeless_2.11:2.3.2 +- com.google.guava:guava:14.0.1 +- com.twitter:hbc-core:2.2.0 +- com.twitter:joauth:6.0.2 +- log4j:log4j:1.2.17 +- net.sf.opencsv:opencsv:2.3 +- org.apache.kafka:kafka-clients:jar:0.10.2.1 +- org.scalanlp:breeze_2.11:0.13 +- org.scalanlp:breeze-macros_2.11:0.13 +- org.typelevel:macro-compat_2.11:1.1.1 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.github.fommil.netlib:core:1.1.2 +- net.sourceforge.f2j:arpack_combined_all:0.1 + +This project bundles the following dependencies under the MIT/X11 license. +See bundled license files for details. + +- org.slf4j:slf4j-log4j12:1.7.15 +- org.spire-math:spire-macros_2.11:0.13.0 +- org.spire-math:spire_2.11:0.13.0 +- org.typelevel:machinist_2.11:0.6.1 + +This project bundles the following dependencies under the Mozilla Public license. + +- com.github.rwl:jtransforms:2.4.0 (https://sourceforge.net/projects/jtransforms/) + +flink-metrics-datadog +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.squareup.okhttp3:okhttp:3.7.0 +- com.squareup.okio:okio:1.12.0 + +flink-dist +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.code.findbugs:jsr305:1.3.9 +- com.twitter:chill-java:0.7.6 +- com.twitter:chill_2.11:0.7.6 +- com.typesafe:config:1.3.0 +- com.typesafe:ssl-config-core_2.11:0.2.1 +- com.typesafe.akka:akka-actor_2.11:2.4.20 +- com.typesafe.akka:akka-camel_2.11:2.4.20 +- com.typesafe.akka:akka-protobuf_2.11:2.4.20 +- com.typesafe.akka:akka-slf4j_2.11:2.4.20 +- com.typesafe.akka:akka-stream_2.11:2.4.20 +- commons-cli:commons-cli:1.3.1 +- commons-collections:commons-collections:3.2.2 +- commons-io:commons-io:2.4 +- org.apache.camel:camel-core:2.17.7 +- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-lang3:3.3.2 +- org.apache.commons:commons-math3:3.5 +- org.javassist:javassist:3.19.0-GA +- org.objenesis:objenesis:2.1 +- org.rocksdb:rocksdbjni:5.7.5 +- org.xerial.snappy:snappy-java:1.1.4 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.esotericsoftware.kryo:kryo:2.24.0 +- com.esotericsoftware.minlog:minlog:1.2 +- org.clapper:grizzled-slf4j_2.11:1.3.2 + +The following dependencies all share the same BSD license which you find under licenses/LICENSE.scala. + +- org.scala-lang:scala-compiler:2.11.12 +- org.scala-lang:scala-library:2.11.12 +- org.scala-lang:scala-reflect:2.11.12 +- org.scala-lang.modules:scala-java8-compat_2.11:0.7.0 +- org.scala-lang.modules:scala-parser-combinators_2.11:1.0.4 +- org.scala-lang.modules:scala-xml_2.11:1.0.5 + +This project bundles the following dependencies under the MIT/X11 license. +See bundled license files for details. + +- com.github.scopt:scopt_2.11:3.5.0 +- org.slf4j:slf4j-api:1.7.15 + +This project bundles "org.tukaani:xz:1.5". +This Java implementation of XZ has been put into the public domain, thus you can do +whatever you want with it. All the files in the package have been written by Lasse Collin, +but some files are heavily based on public domain code written by Igor Pavlov. + +This project bundles the following dependencies under the Creative Commons CC0 "No Rights Reserved". + +- org.reactivestreams:reactive-streams:1.0.0 + +This product includes software from the Spring Framework, +under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +Apache Commons Collections +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +Apache Commons Compress +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons Math +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed for Orekit by +CS Systèmes d'Information (http://www.c-s.fr/) +Copyright 2010-2012 CS Systèmes d'Information + +flink-runtime +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.typesafe.akka:akka-remote_2.11:2.4.20 +- io.netty:netty:3.10.6.Final +- org.apache.zookeeper:zookeeper:3.4.10 +- org.uncommons.maths:uncommons-maths:1.2.2a + +This project bundles io.netty:netty:3.10.6.Final from which it inherits the following notices: + +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * licenses/LICENSE.jsr166y (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.base64 (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.jzlib (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product contains a modified version of 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * licenses/LICENSE.webbit (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +flink-shaded-curator +Copyright 2014-2018 The Apache Software Foundation + +- com.google.guava:guava:16.0.1 +- org.apache.curator:curator-client:2.12.0 +- org.apache.curator:curator-framework:2.12.0 +- org.apache.curator:curator-recipes:2.12.0 + +Curator Recipes +Copyright 2011-2017 The Apache Software Foundation + +Curator Framework +Copyright 2011-2017 The Apache Software Foundation + +Curator Client +Copyright 2011-2017 The Apache Software Foundation + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +flink-shaded-jackson-2 +Copyright 2014-2018 The Apache Software Foundation + +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +Apache Commons CLI +Copyright 2001-2015 The Apache Software Foundation + +flink-mesos +Copyright 2014-2018 The Apache Software Foundation + +- com.netflix.fenzo:fenzo-core:0.10.1 +- org.apache.mesos:mesos:1.0.1 +- com.fasterxml.jackson.core:jackson-annotations:2.4.0 +- com.fasterxml.jackson.core:jackson-core:2.4.5 +- com.fasterxml.jackson.core:jackson-databind:2.4.5 + +- com.google.protobuf:protobuf-java:2.6.2 + +mesos +Copyright 2016 The Apache Software Foundation + +flink-yarn +Copyright 2014-2018 The Apache Software Foundation + +Camel :: Core +Copyright 2007-2017 The Apache Software Foundation + + ========================================================================= + == NOTICE file corresponding to the section 4 d of == + == the Apache License, Version 2.0, == + == in this case for the Apache Camel distribution. == + ========================================================================= + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + Please read the different LICENSE files present in the licenses directory of + this distribution. + +Objenesis +Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita + +flink-table +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.esri.geometry:esri-geometry-api:2.0.0 +- com.google.guava:guava:19.0 +- joda-time:joda-time:2.5 +- net.hydromatic:aggdesigner-algorithm:6.0 +- org.apache.calcite:calcite-core:1.17.0 +- org.apache.calcite:calcite-linq4j:1.17.0 +- org.apache.calcite.avatica:avatica-core:1.12.0 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details + +- org.codehaus.janino:janino:3.0.7 +- org.codehaus.janino:commons-compiler:3.0.7 + +Calcite Core +Copyright 2012-2018 The Apache Software Foundation + +Apache Calcite Avatica +Copyright 2012-2018 The Apache Software Foundation + +Calcite Linq4j +Copyright 2012-2018 The Apache Software Foundation + +============================================================================= += NOTICE file corresponding to section 4d of the Apache License Version 2.0 = +============================================================================= +This product includes software developed by +Joda.org (http://www.joda.org/). + +flink-runtime-web +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- nvd3#1.8.4 + +This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) +See bundled license files for details. + +- angular:1.4.8 +- angular-drag-and-drop-list:1.4.0 +- angular-moment:0.10.3 +- angular-ui-router:0.2.15 +- bootstrap:3.3.6 +- dagre:0.7.5 +- dagre-d3:0.4.17 +- ev-emitter:1.1.1 +- font-awesome:4.5.0 (CSS) +- graphlib:1.0.7 +- imagesloaded:4.1.4 +- jquery:2.2.0 +- lodash:3.10.1 +- moment:2.10.6 +- moment-duration-format:1.3.0 +- qtip2:2.2.1 +- Split.js:1.0.6 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- d3:3.5.12 + +This project bundles the following dependencies under SIL OFL 1.1 license. (https://opensource.org/licenses/OFL-1.1) +See bundled license files for details. + +- font-awesome:4.5.0 (Font) + +flink-swift-fs-hadoop +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hadoop:hadoop-annotations:2.8.1 +- org.apache.hadoop:hadoop-auth:2.8.1 +- org.apache.hadoop:hadoop-client:2.8.1 +- org.apache.hadoop:hadoop-common:2.8.1 +- org.apache.hadoop:hadoop-hdfs:2.8.1 +- org.apache.hadoop:hadoop-hdfs-client:2.8.1 +- org.apache.hadoop:hadoop-openstack:2.8.1 +- org.apache.htrace:htrace-core4:4.0.1-incubating +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-math3:3.5 +- commons-beanutils:commons-beanutils:1.8.3 +- commons-cli:commons-cli:1.3.1 +- commons-codec:commons-codec:1.10 +- commons-collections:commons-collections:3.2.2 +- commons-configuration:commons-configuration:1.7 +- commons-digester:commons-digester:1.8.1 +- commons-io:commons-io:2.4 +- commons-lang:commons-lang:2.6 +- commons-logging:commons-logging:1.1.3 +- commons-net:commons-net:3.1 +- commons-httpclient:commons-httpclient:3.1 +- com.google.guava:guava:11.0.2 +- com.google.code.gson:gson:2.2.4 +- com.squareup.okhttp:okhttp:2.4.0 +- com.squareup.okio:okio:1.4.0 +- net.minidev:json-smart:1.1.1 +- com.nimbusds:nimbus-jose-jwt:3.9 +- org.mortbay.jetty:jetty-sslengine:6.1.26 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 + +This project bundles the following dependencies under the BSD License. +See bundled license files for details. + +- xmlenc:xmlenc:0.52 +- com.google.protobuf:protobuf-java:2.5.0 + +This project bundles "net.jcip:jcip-annotations:1.0". +Written by Brian Goetz and Tim Peierls with assistance from members of JCP JSR-166 Expert Group +and released to the public domain, as explained by the Creative Commons public domain license. +https://creativecommons.org/licenses/publicdomain/ + +This project bundles "org.tukaani:xz:1.5". +This Java implementation of XZ has been put into the public domain, thus you can do +whatever you want with it. All the files in the package have been written by Lasse Collin, +but some files are heavily based on public domain code written by Igor Pavlov. + +This project bundles org.apache.hadoop:*:2.8.1 from which it inherits the following notices: + +The Apache Hadoop project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + +For the org.apache.hadoop.util.bloom.* classes: + +/** + * + * Copyright (c) 2005, European Commission project OneLab under contract + * 034819 (http://www.one-lab.org) + * All rights reserved. + * Redistribution and use in source and binary forms, with or + * without modification, are permitted provided that the following + * conditions are met: + * - Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * - Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the distribution. + * - Neither the name of the University Catholique de Louvain - UCL + * nor the names of its contributors may be used to endorse or + * promote products derived from this software without specific prior + * written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN + * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +For portions of the native implementation of slicing-by-8 CRC calculation +in src/main/native/src/org/apache/hadoop/util: + +/** + * Copyright 2008,2009,2010 Massachusetts Institute of Technology. + * All rights reserved. Use of this source code is governed by a + * BSD-style license that can be found in the LICENSE file. + */ + +For src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,lz4hc.c}, + +/* + LZ4 - Fast LZ compression algorithm + Header File + Copyright (C) 2011-2014, Yann Collet. + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - LZ4 source repository : http://code.google.com/p/lz4/ + - LZ4 public forum : https://groups.google.com/forum/#!forum/lz4c +*/ + +For hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/util/tree.h +--------------------------------------------------------------------- +Copyright 2002 Niels Provos +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles binaries of leveldbjni +(https://github.com/fusesource/leveldbjni), which is available under the +following license: + +Copyright (c) 2011 FuseSource Corp. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of FuseSource Corp. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/compat/{fstatat|openat|unlinkat}.h: + +Copyright (c) 2012 The FreeBSD Foundation +All rights reserved. + +This software was developed by Pawel Jakub Dawidek under sponsorship from +the FreeBSD Foundation. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHORS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHORS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS +OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + +============= + +The binary distribution of this product bundles binaries of leveldb +(http://code.google.com/p/leveldb/), which is available under the following +license: + +Copyright (c) 2011 The LevelDB Authors. All rights reserved. + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +The binary distribution of this product bundles binaries of snappy +(http://code.google.com/p/snappy/), which is available under the following +license: + +Copyright 2011, Google Inc. +All rights reserved. + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/ +-------------------------------------------------------------------------------- +Copyright (C) 2008-2016, SpryMedia Ltd. + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2010 Aleksander Williams + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2011-2016 Tim Wood, Iskren Chernev, Moment.js contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2 +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css +And the binary distribution of this product bundles these dependencies under the +following license: +Mockito 1.8.5 +SLF4J 1.7.10 +-------------------------------------------------------------------------------- + +The MIT License (MIT) + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery +-------------------------------------------------------------------------------- + +Copyright jQuery Foundation and other contributors, https://jquery.org/ + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/jquery/jquery + +The following license applies to all parts of this software except as +documented below: + +==== + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +All files located in the node_modules and external directories are +externally maintained libraries used by this software which have their +own licenses; we recommend you read them, as their terms may differ from +the terms above. + +For: +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js.gz +-------------------------------------------------------------------------------- + +Copyright (c) 2014 Ivan Bozhanov + +For: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js +-------------------------------------------------------------------------------- + +D3 is available under a 3-clause BSD license. For details, see: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE + +The binary distribution of this product bundles these dependencies under the +following license: +HSQLDB Database 2.0.0 +-------------------------------------------------------------------------------- +"COPYRIGHTS AND LICENSES (based on BSD License) + +For work developed by the HSQL Development Group: + +Copyright (c) 2001-2016, The HSQL Development Group +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +Neither the name of the HSQL Development Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +For work originally developed by the Hypersonic SQL Group: + +Copyright (c) 1995-2000 by the Hypersonic SQL Group. +All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Neither the name of the Hypersonic SQL Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +This software consists of voluntary contributions made by many individuals on behalf of the +Hypersonic SQL Group." + +The binary distribution of this product bundles these dependencies under the +following license: +servlet-api 2.5 +jsp-api 2.1 +Streaming API for XML 1.0 +-------------------------------------------------------------------------------- +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 +1. Definitions.  + +1.1. Contributor means each individual or entity +that creates or contributes to the creation of +Modifications.  + +1.2. Contributor Version means the combination of the +Original Software, prior Modifications used by a Contributor (if any), and the +Modifications made by that particular Contributor.  + +1.3. Covered +Software means (a) the Original Software, or (b) Modifications, or (c) the +combination of files containing Original Software with files containing +Modifications, in each case including portions +thereof.  + +1.4. Executable means the Covered Software in any form other +than Source Code.  + +1.5. Initial Developer means the individual or entity +that first makes Original Software available under this +License.  + +1.6. Larger Work means a work which combines Covered Software or +portions thereof with code not governed by the terms of this +License.  + +1.7. License means this document.  + +1.8. Licensable means +having the right to grant, to the maximum extent possible, whether at the time +of the initial grant or subsequently acquired, any and all of the rights +conveyed herein.  + +1.9. Modifications means the Source Code and Executable +form of any of the following: +A. Any file that results from an addition to, +deletion from or modification of the contents of a file containing Original +Software or previous Modifications; +B. Any new file that contains any part of the Original Software +or previous Modification; or +C. Any new file that is contributed or otherwise made available +under the terms of this License.  + +1.10. Original Software means the Source Code and Executable form of +computer software code that is originally released under this License.  + +1.11. Patent Claims means any patent claim(s), now owned or +hereafter acquired, including without limitation, method, process, and apparatus +claims, in any patent Licensable by grantor.  + +1.12. Source Code means (a) the common form of computer software code in which +modifications are made and (b) associated documentation included in or +with such code.  + +1.13. You (or Your) means an individual or a legal entity exercising rights +under, and complying with all of the terms of, this License. For legal entities, +You includes any entity which controls, is controlled by, or is under common control +with You. For purposes of this definition, control means (a) the power, direct +or indirect, to cause the direction or management of such entity, whether by +contract or otherwise, or (b) ownership of more than fifty percent (50%) of the +outstanding shares or beneficial ownership of such entity.  + +2. License Grants. + +2.1. The Initial Developer Grant. Conditioned upon Your compliance +with Section 3.1 below and subject to third party intellectual property claims, +the Initial Developer hereby grants You a world-wide, royalty-free, +non-exclusive license:  + +(a) under intellectual property rights (other than +patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, +display, perform, sublicense and distribute the Original Software (or portions +thereof), with or without Modifications, and/or as part of a Larger Work; +and  + +(b) under Patent Claims infringed by the making, using or selling of +Original Software, to make, have made, use, practice, sell, and offer for sale, +and/or otherwise dispose of the Original Software (or portions +thereof); + +(c) The licenses granted in Sections 2.1(a) and (b) are +effective on the date Initial Developer first distributes or otherwise makes the +Original Software available to a third party under the terms of this +License; + +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or +(ii) the combination of the Original Software with other software or +devices.  + +2.2. Contributor Grant. Conditioned upon Your compliance with +Section 3.1 below and subject to third party intellectual property claims, each +Contributor hereby grants You a world-wide, royalty-free, non-exclusive +license:  + +(a) under intellectual property rights (other than patent or +trademark) Licensable by Contributor to use, reproduce, modify, display, +perform, sublicense and distribute the Modifications created by such Contributor +(or portions thereof), either on an unmodified basis, with other Modifications, +as Covered Software and/or as part of a Larger Work; and  + +(b) under Patent +Claims infringed by the making, using, or selling of Modifications made by that +Contributor either alone and/or in combination with its Contributor Version (or +portions of such combination), to make, use, sell, offer for sale, have made, +and/or otherwise dispose of: (1) Modifications made by that Contributor (or +portions thereof); and (2) the combination of Modifications made by that +Contributor with its Contributor Version (or portions of such +combination).  + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are +effective on the date Contributor first distributes or otherwise makes the +Modifications available to a third party. + +(d) Notwithstanding Section 2.2(b) +above, no patent license is granted: (1) for any code that Contributor has +deleted from the Contributor Version; (2) for infringements caused by: +(i) third party modifications of Contributor Version, or (ii) the combination +of Modifications made by that Contributor with other software (except as part of +the Contributor Version) or other devices; or (3) under Patent Claims infringed +by Covered Software in the absence of Modifications made by that +Contributor.  + +3. Distribution Obligations.  + +3.1. Availability of Source +Code. Any Covered Software that You distribute or otherwise make available in +Executable form must also be made available in Source Code form and that Source +Code form must be distributed only under the terms of this License. You must +include a copy of this License with every copy of the Source Code form of the +Covered Software You distribute or otherwise make available. You must inform +recipients of any such Covered Software in Executable form as to how they can +obtain such Covered Software in Source Code form in a reasonable manner on or +through a medium customarily used for software exchange.  + +3.2. +Modifications. The Modifications that You create or to which You contribute are +governed by the terms of this License. You represent that You believe Your +Modifications are Your original creation(s) and/or You have sufficient rights to +grant the rights conveyed by this License.  + +3.3. Required Notices. You must +include a notice in each of Your Modifications that identifies You as the +Contributor of the Modification. You may not remove or alter any copyright, +patent or trademark notices contained within the Covered Software, or any +notices of licensing or any descriptive text giving attribution to any +Contributor or the Initial Developer.  + +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source Code +form that alters or restricts the applicable version of this License or the +recipients rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. + +3.5. Distribution of Executable Versions. You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipients rights in the Source Code form from the rights set forth in this +License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer.  + +3.6. Larger Works. You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software.  + +4. Versions of the License.  + +4.1. +New Versions. Sun Microsystems, Inc. is the initial license steward and may +publish revised and/or new versions of this License from time to time. Each +version will be given a distinguishing version number. Except as provided in +Section 4.3, no one other than the license steward has the right to modify this +License.  + +4.2. Effect of New Versions. You may always continue to use, +distribute or otherwise make the Covered Software available under the terms of +the version of the License under which You originally received the Covered +Software. If the Initial Developer includes a notice in the Original Software +prohibiting it from being distributed or otherwise made available under any +subsequent version of the License, You must distribute and make the Covered +Software available under the terms of the version of the License under which You +originally received the Covered Software. Otherwise, You may also choose to use, +distribute or otherwise make the Covered Software available under the terms of +any subsequent version of the License published by the license +steward.  + +4.3. Modified Versions. When You are an Initial Developer and You +want to create a new license for Your Original Software, You may create and use +a modified version of this License if You: (a) rename the license and remove +any references to the name of the license steward (except to note that the +license differs from this License); and (b) otherwise make it clear that the +license contains terms which differ from this License.  + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, +WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT +LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, +MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY +COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER +OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR +CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS +LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER +THIS DISCLAIMER.  + +6. TERMINATION.  + +6.1. This License and the rights +granted hereunder will terminate automatically if You fail to comply with terms +herein and fail to cure such breach within 30 days of becoming aware of the +breach. Provisions which, by their nature, must remain in effect beyond the +termination of this License shall survive.  + +6.2. If You assert a patent +infringement claim (excluding declaratory judgment actions) against Initial +Developer or a Contributor (the Initial Developer or Contributor against whom +You assert such claim is referred to as Participant) alleging that the +Participant Software (meaning the Contributor Version where the Participant is a +Contributor or the Original Software where the Participant is the Initial +Developer) directly or indirectly infringes any patent, then any and all rights +granted directly or indirectly to You by such Participant, the Initial Developer +(if the Initial Developer is not the Participant) and all Contributors under +Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from +Participant terminate prospectively and automatically at the expiration of such +60 day notice period, unless if within such 60 day period You withdraw Your +claim with respect to the Participant Software against such Participant either +unilaterally or pursuant to a written agreement with Participant.  + +6.3. In +the event of termination under Sections 6.1 or 6.2 above, all end user licenses +that have been validly granted by You or any distributor hereunder prior to +termination (excluding licenses granted to You by any distributor) shall survive +termination.  + +7. LIMITATION OF LIABILITY. +UNDER NO CIRCUMSTANCES AND UNDER +NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, +SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF +COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY +PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY +CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF +GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER +COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE +POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO +LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO +THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT +ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO +THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.  + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a commercial item, as that term is defined in +48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as +that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer +software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. +1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through +227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software +with only those rights set forth herein. This U.S. Government Rights clause is +in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision +that addresses Government rights in computer software under this +License.  + +9. MISCELLANEOUS. +This License represents the complete agreement +concerning subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdictions conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software.  + +10. RESPONSIBILITY FOR CLAIMS. +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability.  + +The binary distribution of this product bundles these dependencies under the +following license: +Jersey 1.9 +JAXB API bundle for GlassFish V3 2.2.2 +JAXB RI 2.2.3 +-------------------------------------------------------------------------------- +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 + +1. Definitions. + +1.1. “Contributor” means each individual or entity that creates or +contributes to the creation of Modifications. +1.2. “Contributor Version” means the combination of the Original Software, +prior Modifications used by a Contributor (if any), and the Modifications made +by that particular Contributor. +1.3. “Covered Software” means (a) the Original Software, or (b) +Modifications, or (c) the combination of files containing Original Software with +files containing Modifications, in each case including portions thereof. +1.4. “Executable” means the Covered Software in any form other than Source +Code. +1.5. “Initial Developer” means the individual or entity that first makes +Original Software available under this License. +1.6. “Larger Work” means a work which combines Covered Software or portions +thereof with code not governed by the terms of this License. +1.7. “License” means this document. +1.8. “Licensable” means having the right to grant, to the maximum extent +possible, whether at the time of the initial grant or subsequently acquired, any +and all of the rights conveyed herein. +1.9. “Modifications” means the Source Code and Executable form of any of the +following: +A. Any file that results from an addition to, deletion from or modification of +the contents of a file containing Original Software or previous Modifications; +B. Any new file that contains any part of the Original Software or previous +Modification; or +C. Any new file that is contributed or otherwise made available under the terms +of this License. +1.10. “Original Software” means the Source Code and Executable form of +computer software code that is originally released under this License. +1.11. “Patent Claims” means any patent claim(s), now owned or hereafter +acquired, including without limitation, method, process, and apparatus claims, +in any patent Licensable by grantor. +1.12. “Source Code” means (a) the common form of computer software code in +which modifications are made and (b) associated documentation included in or +with such code. +1.13. “You” (or “Your”) means an individual or a legal entity exercising +rights under, and complying with all of the terms of, this License. For legal +entities, “You” includes any entity which controls, is controlled by, or is +under common control with You. For purposes of this definition, “control” +means (a) the power, direct or indirect, to cause the direction or management of +such entity, whether by contract or otherwise, or (b) ownership of more than +fifty percent (50%) of the outstanding shares or beneficial ownership of such +entity. + +2.1. The Initial Developer Grant. + +Conditioned upon Your compliance with Section 3.1 below and subject to +third party intellectual property claims, the Initial Developer hereby grants +You a world-wide, royalty-free, non-exclusive license: +(a) under intellectual +property rights (other than patent or trademark) Licensable by Initial +Developer, to use, reproduce, modify, display, perform, sublicense and +distribute the Original Software (or portions thereof), with or without +Modifications, and/or as part of a Larger Work; and +(b) under Patent Claims +infringed by the making, using or selling of Original Software, to make, have +made, use, practice, sell, and offer for sale, and/or otherwise dispose of the +Original Software (or portions thereof). +(c) The licenses granted in Sections +2.1(a) and (b) are effective on the date Initial Developer first distributes or +otherwise makes the Original Software available to a third party under the terms +of this License. +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or (ii) +the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. + +Conditioned upon Your compliance with Section 3.1 below and +subject to third party intellectual property claims, each Contributor hereby +grants You a world-wide, royalty-free, non-exclusive license: +(a) under +intellectual property rights (other than patent or trademark) Licensable by +Contributor to use, reproduce, modify, display, perform, sublicense and +distribute the Modifications created by such Contributor (or portions thereof), +either on an unmodified basis, with other Modifications, as Covered Software +and/or as part of a Larger Work; and +(b) under Patent Claims infringed by the +making, using, or selling of Modifications made by that Contributor either alone +and/or in combination with its Contributor Version (or portions of such +combination), to make, use, sell, offer for sale, have made, and/or otherwise +dispose of: (1) Modifications made by that Contributor (or portions thereof); +and (2) the combination of Modifications made by that Contributor with its +Contributor Version (or portions of such combination). +(c) The licenses granted +in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first +distributes or otherwise makes the Modifications available to a third +party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: +(1) for any code that Contributor has deleted from the Contributor Version; (2) +for infringements caused by: (i) third party modifications of Contributor +Version, or (ii) the combination of Modifications made by that Contributor with +other software (except as part of the Contributor Version) or other devices; or +(3) under Patent Claims infringed by Covered Software in the absence of +Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. +Any Covered Software that You distribute or +otherwise make available in Executable form must also be made available in +Source Code form and that Source Code form must be distributed only under the +terms of this License. You must include a copy of this License with every copy +of the Source Code form of the Covered Software You distribute or otherwise make +available. You must inform recipients of any such Covered Software in Executable +form as to how they can obtain such Covered Software in Source Code form in a +reasonable manner on or through a medium customarily used for software +exchange. +3.2. Modifications. +The Modifications that You create or to which +You contribute are governed by the terms of this License. You represent that You +believe Your Modifications are Your original creation(s) and/or You have +sufficient rights to grant the rights conveyed by this License. +3.3. Required Notices. +You must include a notice in each of Your Modifications that +identifies You as the Contributor of the Modification. You may not remove or +alter any copyright, patent or trademark notices contained within the Covered +Software, or any notices of licensing or any descriptive text giving attribution +to any Contributor or the Initial Developer. +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source +Code form that alters or restricts the applicable version of this License or the +recipients' rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. +3.5. Distribution of Executable Versions. +You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipient's rights in the Source Code form from the rights set forth in +this License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer. +3.6. Larger Works. +You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. +Oracle is the initial license steward and may publish revised and/or +new versions of this License from time to time. Each version will be given a +distinguishing version number. Except as provided in Section 4.3, no one other +than the license steward has the right to modify this License. +4.2. Effect of New Versions. +You may always continue to use, distribute or otherwise make the +Covered Software available under the terms of the version of the License under +which You originally received the Covered Software. If the Initial Developer +includes a notice in the Original Software prohibiting it from being distributed +or otherwise made available under any subsequent version of the License, You +must distribute and make the Covered Software available under the terms of the +version of the License under which You originally received the Covered Software. +Otherwise, You may also choose to use, distribute or otherwise make the Covered +Software available under the terms of any subsequent version of the License +published by the license steward. +4.3. Modified Versions. +When You are an +Initial Developer and You want to create a new license for Your Original +Software, You may create and use a modified version of this License if You: (a) +rename the license and remove any references to the name of the license steward +(except to note that the license differs from this License); and (b) otherwise +make it clear that the license contains terms which differ from this +License. + +COVERED SOFTWARE IS PROVIDED UNDER THIS +LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE +IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR +NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED +SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY +RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE +COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF +WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED +SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will +terminate automatically if You fail to comply with terms herein and fail to cure +such breach within 30 days of becoming aware of the breach. Provisions which, by +their nature, must remain in effect beyond the termination of this License shall +survive. +6.2. If You assert a patent infringement claim (excluding declaratory +judgment actions) against Initial Developer or a Contributor (the Initial +Developer or Contributor against whom You assert such claim is referred to as +“Participant”) alleging that the Participant Software (meaning the +Contributor Version where the Participant is a Contributor or the Original +Software where the Participant is the Initial Developer) directly or indirectly +infringes any patent, then any and all rights granted directly or indirectly to +You by such Participant, the Initial Developer (if the Initial Developer is not +the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this +License shall, upon 60 days notice from Participant terminate prospectively and +automatically at the expiration of such 60 day notice period, unless if within +such 60 day period You withdraw Your claim with respect to the Participant +Software against such Participant either unilaterally or pursuant to a written +agreement with Participant. +6.3. If You assert a patent infringement claim +against Participant alleging that the Participant Software directly or +indirectly infringes any patent where such claim is resolved (such as by license +or settlement) prior to the initiation of patent infringement litigation, then +the reasonable value of the licenses granted by such Participant under Sections +2.1 or 2.2 shall be taken into account in determining the amount or value of any +payment or license. +6.4. In the event of termination under Sections 6.1 or 6.2 +above, all end user licenses that have been validly granted by You or any +distributor hereunder prior to termination (excluding licenses granted to You by +any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT +(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL +DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY +SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, +SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, +WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER +FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN +IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS +LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL +INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW +PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR +LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND +LIMITATION MAY NOT APPLY TO YOU. + +The Covered +Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 +(Oct. 1995), consisting of “commercial computer software” (as that term is +defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software +documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). +Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 +(June 1995), all U.S. Government End Users acquire Covered Software with only +those rights set forth herein. This U.S. Government Rights clause is in lieu of, +and supersedes, any other FAR, DFAR, or other clause or provision that addresses +Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning +subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdiction's conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys' fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability. + +The binary distribution of this product bundles these dependencies under the +following license: +Protocol Buffer Java API 2.5.0 +-------------------------------------------------------------------------------- +This license applies to all parts of Protocol Buffers except the following: + + - Atomicops support for generic gcc, located in + src/google/protobuf/stubs/atomicops_internals_generic_gcc.h. + This file is copyrighted by Red Hat Inc. + + - Atomicops support for AIX/POWER, located in + src/google/protobuf/stubs/atomicops_internals_power.h. + This file is copyrighted by Bloomberg Finance LP. + +Copyright 2014, Google Inc. All rights reserved. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +For: +XML Commons External Components XML APIs 1.3.04 +-------------------------------------------------------------------------------- +By obtaining, using and/or copying this work, you (the licensee) agree that you +have read, understood, and will comply with the following terms and conditions. + +Permission to copy, modify, and distribute this software and its documentation, +with or without modification, for any purpose and without fee or royalty is +hereby granted, provided that you include the following on ALL copies of the +software and documentation or portions thereof, including modifications: +- The full text of this NOTICE in a location viewable to users of the +redistributed or derivative work. +- Any pre-existing intellectual property disclaimers, notices, or terms and +conditions. If none exist, the W3C Software Short Notice should be included +(hypertext is preferred, text is permitted) within the body of any redistributed +or derivative code. +- Notice of any changes or modifications to the files, including the date changes +were made. (We recommend you provide URIs to the location from which the code is +derived.) + +The binary distribution of this product bundles these dependencies under the +following license: +JUnit 4.11 +ecj-4.3.1.jar +-------------------------------------------------------------------------------- +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC +LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM +CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation +distributed under this Agreement, and +b) in the case of each subsequent Contributor: +i) changes to the Program, and +ii) additions to the Program; +where such changes and/or additions to the Program originate from and are +distributed by that particular Contributor. A Contribution 'originates' from a +Contributor if it was added to the Program by such Contributor itself or anyone +acting on such Contributor's behalf. Contributions do not include additions to +the Program which: (i) are separate modules of software distributed in +conjunction with the Program under their own license agreement, and (ii) are not +derivative works of the Program. +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are +necessarily infringed by the use or sale of its Contribution alone or when +combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, +including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free copyright license to +reproduce, prepare derivative works of, publicly display, publicly perform, +distribute and sublicense the Contribution of such Contributor, if any, and such +derivative works, in source code and object code form. +b) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed +Patents to make, use, sell, offer to sell, import and otherwise transfer the +Contribution of such Contributor, if any, in source code and object code form. +This patent license shall apply to the combination of the Contribution and the +Program if, at the time the Contribution is added by the Contributor, such +addition of the Contribution causes such combination to be covered by the +Licensed Patents. The patent license shall not apply to any other combinations +which include the Contribution. No hardware per se is licensed hereunder. +c) Recipient understands that although each Contributor grants the licenses to +its Contributions set forth herein, no assurances are provided by any +Contributor that the Program does not infringe the patent or other intellectual +property rights of any other entity. Each Contributor disclaims any liability to +Recipient for claims brought by any other entity based on infringement of +intellectual property rights or otherwise. As a condition to exercising the +rights and licenses granted hereunder, each Recipient hereby assumes sole +responsibility to secure any other intellectual property rights needed, if any. +For example, if a third party patent license is required to allow Recipient to +distribute the Program, it is Recipient's responsibility to acquire that license +before distributing the Program. +d) Each Contributor represents that to its knowledge it has sufficient copyright +rights in its Contribution, if any, to grant the copyright license set forth in +this Agreement. +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its +own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and +b) its license agreement: +i) effectively disclaims on behalf of all Contributors all warranties and +conditions, express and implied, including warranties or conditions of title and +non-infringement, and implied warranties or conditions of merchantability and +fitness for a particular purpose; +ii) effectively excludes on behalf of all Contributors all liability for +damages, including direct, indirect, special, incidental and consequential +damages, such as lost profits; +iii) states that any provisions which differ from this Agreement are offered by +that Contributor alone and not by any other party; and +iv) states that source code for the Program is available from such Contributor, +and informs licensees how to obtain it in a reasonable manner on or through a +medium customarily used for software exchange. +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and +b) a copy of this Agreement must be included with each copy of the Program. +Contributors may not remove or alter any copyright notices contained within the +Program. + +Each Contributor must identify itself as the originator of its Contribution, if +any, in a manner that reasonably allows subsequent Recipients to identify the +originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with +respect to end users, business partners and the like. While this license is +intended to facilitate the commercial use of the Program, the Contributor who +includes the Program in a commercial product offering should do so in a manner +which does not create potential liability for other Contributors. Therefore, if +a Contributor includes the Program in a commercial product offering, such +Contributor ("Commercial Contributor") hereby agrees to defend and indemnify +every other Contributor ("Indemnified Contributor") against any losses, damages +and costs (collectively "Losses") arising from claims, lawsuits and other legal +actions brought by a third party against the Indemnified Contributor to the +extent caused by the acts or omissions of such Commercial Contributor in +connection with its distribution of the Program in a commercial product +offering. The obligations in this section do not apply to any claims or Losses +relating to any actual or alleged intellectual property infringement. In order +to qualify, an Indemnified Contributor must: a) promptly notify the Commercial +Contributor in writing of such claim, and b) allow the Commercial Contributor to +control, and cooperate with the Commercial Contributor in, the defense and any +related settlement negotiations. The Indemnified Contributor may participate in +any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product +offering, Product X. That Contributor is then a Commercial Contributor. If that +Commercial Contributor then makes performance claims, or offers warranties +related to Product X, those performance claims and warranties are such +Commercial Contributor's responsibility alone. Under this section, the +Commercial Contributor would have to defend claims against the other +Contributors related to those performance claims and warranties, and if a court +requires any other Contributor to pay any damages as a result, the Commercial +Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR +IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, +NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each +Recipient is solely responsible for determining the appropriateness of using and +distributing the Program and assumes all risks associated with its exercise of +rights under this Agreement , including but not limited to the risks and costs +of program errors, compliance with applicable laws, damage to or loss of data, +programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY +CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST +PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS +GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this Agreement, and without further action by the parties hereto, such +provision shall be reformed to the minimum extent necessary to make such +provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a +cross-claim or counterclaim in a lawsuit) alleging that the Program itself +(excluding combinations of the Program with other software or hardware) +infringes such Recipient's patent(s), then such Recipient's rights granted under +Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to +comply with any of the material terms or conditions of this Agreement and does +not cure such failure in a reasonable period of time after becoming aware of +such noncompliance. If all Recipient's rights under this Agreement terminate, +Recipient agrees to cease use and distribution of the Program as soon as +reasonably practicable. However, Recipient's obligations under this Agreement +and any licenses granted by Recipient relating to the Program shall continue and +survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in +order to avoid inconsistency the Agreement is copyrighted and may only be +modified in the following manner. The Agreement Steward reserves the right to +publish new versions (including revisions) of this Agreement from time to time. +No one other than the Agreement Steward has the right to modify this Agreement. +The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation +may assign the responsibility to serve as the Agreement Steward to a suitable +separate entity. Each new version of the Agreement will be given a +distinguishing version number. The Program (including Contributions) may always +be distributed subject to the version of the Agreement under which it was +received. In addition, after a new version of the Agreement is published, +Contributor may elect to distribute the Program (including its Contributions) +under the new version. Except as expressly stated in Sections 2(a) and 2(b) +above, Recipient receives no rights or licenses to the intellectual property of +any Contributor under this Agreement, whether expressly, by implication, +estoppel or otherwise. All rights in the Program not expressly granted under +this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the +intellectual property laws of the United States of America. No party to this +Agreement will bring a legal action under this Agreement more than one year +after the cause of action arose. Each party waives its rights to a jury trial in +any resulting litigation. + +The binary distribution of this product bundles these dependencies under the +following license: +ASM Core 3.2 +JSch 0.1.51 +ParaNamer Core 2.3 +JLine 0.9.94 +leveldbjni-all 1.8 +Hamcrest Core 1.3 +xmlenc Library 0.52 +-------------------------------------------------------------------------------- +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles these dependencies under the +following license: +FindBugs-jsr305 3.0.0 +-------------------------------------------------------------------------------- +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The views and conclusions contained in the software and documentation are those +of the authors and should not be interpreted as representing official policies, +either expressed or implied, of the FreeBSD Project. + +The binary distribution of this product bundles these dependencies under the +following license: +Java Concurrency in Practice book annotations 1.0 +-------------------------------------------------------------------------------- +THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE COMMONS +PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY COPYRIGHT AND/OR +OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS AUTHORIZED UNDER THIS +LICENSE OR COPYRIGHT LAW IS PROHIBITED. + +BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE TO BE +BOUND BY THE TERMS OF THIS LICENSE. THE LICENSOR GRANTS YOU THE RIGHTS CONTAINED +HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND CONDITIONS. + +1. Definitions + +"Collective Work" means a work, such as a periodical issue, anthology or +encyclopedia, in which the Work in its entirety in unmodified form, along with a +number of other contributions, constituting separate and independent works in +themselves, are assembled into a collective whole. A work that constitutes a +Collective Work will not be considered a Derivative Work (as defined below) for +the purposes of this License. +"Derivative Work" means a work based upon the Work or upon the Work and other +pre-existing works, such as a translation, musical arrangement, dramatization, +fictionalization, motion picture version, sound recording, art reproduction, +abridgment, condensation, or any other form in which the Work may be recast, +transformed, or adapted, except that a work that constitutes a Collective Work +will not be considered a Derivative Work for the purpose of this License. For +the avoidance of doubt, where the Work is a musical composition or sound +recording, the synchronization of the Work in timed-relation with a moving image +("synching") will be considered a Derivative Work for the purpose of this +License. +"Licensor" means the individual or entity that offers the Work under the terms +of this License. +"Original Author" means the individual or entity who created the Work. +"Work" means the copyrightable work of authorship offered under the terms of +this License. +"You" means an individual or entity exercising rights under this License who has +not previously violated the terms of this License with respect to the Work, or +who has received express permission from the Licensor to exercise rights under +this License despite a previous violation. +2. Fair Use Rights. Nothing in this license is intended to reduce, limit, or +restrict any rights arising from fair use, first sale or other limitations on +the exclusive rights of the copyright owner under copyright law or other +applicable laws. + +3. License Grant. Subject to the terms and conditions of this License, Licensor +hereby grants You a worldwide, royalty-free, non-exclusive, perpetual (for the +duration of the applicable copyright) license to exercise the rights in the Work +as stated below: + +to reproduce the Work, to incorporate the Work into one or more Collective +Works, and to reproduce the Work as incorporated in the Collective Works; +to create and reproduce Derivative Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission the Work including as +incorporated in Collective Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission Derivative Works. +For the avoidance of doubt, where the work is a musical composition: + +Performance Royalties Under Blanket Licenses. Licensor waives the exclusive +right to collect, whether individually or via a performance rights society (e.g. +ASCAP, BMI, SESAC), royalties for the public performance or public digital +performance (e.g. webcast) of the Work. +Mechanical Rights and Statutory Royalties. Licensor waives the exclusive right +to collect, whether individually or via a music rights agency or designated +agent (e.g. Harry Fox Agency), royalties for any phonorecord You create from the +Work ("cover version") and distribute, subject to the compulsory license created +by 17 USC Section 115 of the US Copyright Act (or the equivalent in other +jurisdictions). +Webcasting Rights and Statutory Royalties. For the avoidance of doubt, where the +Work is a sound recording, Licensor waives the exclusive right to collect, +whether individually or via a performance-rights society (e.g. SoundExchange), +royalties for the public digital performance (e.g. webcast) of the Work, subject +to the compulsory license created by 17 USC Section 114 of the US Copyright Act +(or the equivalent in other jurisdictions). +The above rights may be exercised in all media and formats whether now known or +hereafter devised. The above rights include the right to make such modifications +as are technically necessary to exercise the rights in other media and formats. +All rights not expressly granted by Licensor are hereby reserved. + +4. Restrictions.The license granted in Section 3 above is expressly made subject +to and limited by the following restrictions: + +You may distribute, publicly display, publicly perform, or publicly digitally +perform the Work only under the terms of this License, and You must include a +copy of, or the Uniform Resource Identifier for, this License with every copy or +phonorecord of the Work You distribute, publicly display, publicly perform, or +publicly digitally perform. You may not offer or impose any terms on the Work +that alter or restrict the terms of this License or the recipients' exercise of +the rights granted hereunder. You may not sublicense the Work. You must keep +intact all notices that refer to this License and to the disclaimer of +warranties. You may not distribute, publicly display, publicly perform, or +publicly digitally perform the Work with any technological measures that control +access or use of the Work in a manner inconsistent with the terms of this +License Agreement. The above applies to the Work as incorporated in a Collective +Work, but this does not require the Collective Work apart from the Work itself +to be made subject to the terms of this License. If You create a Collective +Work, upon notice from any Licensor You must, to the extent practicable, remove +from the Collective Work any credit as required by clause 4(b), as requested. If +You create a Derivative Work, upon notice from any Licensor You must, to the +extent practicable, remove from the Derivative Work any credit as required by +clause 4(b), as requested. +If you distribute, publicly display, publicly perform, or publicly digitally +perform the Work or any Derivative Works or Collective Works, You must keep +intact all copyright notices for the Work and provide, reasonable to the medium +or means You are utilizing: (i) the name of the Original Author (or pseudonym, +if applicable) if supplied, and/or (ii) if the Original Author and/or Licensor +designate another party or parties (e.g. a sponsor institute, publishing entity, +journal) for attribution in Licensor's copyright notice, terms of service or by +other reasonable means, the name of such party or parties; the title of the Work +if supplied; to the extent reasonably practicable, the Uniform Resource +Identifier, if any, that Licensor specifies to be associated with the Work, +unless such URI does not refer to the copyright notice or licensing information +for the Work; and in the case of a Derivative Work, a credit identifying the use +of the Work in the Derivative Work (e.g., "French translation of the Work by +Original Author," or "Screenplay based on original Work by Original Author"). +Such credit may be implemented in any reasonable manner; provided, however, that +in the case of a Derivative Work or Collective Work, at a minimum such credit +will appear where any other comparable authorship credit appears and in a manner +at least as prominent as such other comparable authorship credit. +5. Representations, Warranties and Disclaimer + +UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR OFFERS +THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY KIND CONCERNING +THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE, INCLUDING, WITHOUT +LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY, FITNESS FOR A PARTICULAR +PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF LATENT OR OTHER DEFECTS, ACCURACY, +OR THE PRESENCE OF ABSENCE OF ERRORS, WHETHER OR NOT DISCOVERABLE. SOME +JURISDICTIONS DO NOT ALLOW THE EXCLUSION OF IMPLIED WARRANTIES, SO SUCH +EXCLUSION MAY NOT APPLY TO YOU. + +6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE LAW, IN +NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR ANY SPECIAL, +INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES ARISING OUT OF THIS +LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + +7. Termination + +This License and the rights granted hereunder will terminate automatically upon +any breach by You of the terms of this License. Individuals or entities who have +received Derivative Works or Collective Works from You under this License, +however, will not have their licenses terminated provided such individuals or +entities remain in full compliance with those licenses. Sections 1, 2, 5, 6, 7, +and 8 will survive any termination of this License. +Subject to the above terms and conditions, the license granted here is perpetual +(for the duration of the applicable copyright in the Work). Notwithstanding the +above, Licensor reserves the right to release the Work under different license +terms or to stop distributing the Work at any time; provided, however that any +such election will not serve to withdraw this License (or any other license that +has been, or is required to be, granted under the terms of this License), and +this License will continue in full force and effect unless terminated as stated +above. +8. Miscellaneous + +Each time You distribute or publicly digitally perform the Work or a Collective +Work, the Licensor offers to the recipient a license to the Work on the same +terms and conditions as the license granted to You under this License. +Each time You distribute or publicly digitally perform a Derivative Work, +Licensor offers to the recipient a license to the original Work on the same +terms and conditions as the license granted to You under this License. +If any provision of this License is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this License, and without further action by the parties to this +agreement, such provision shall be reformed to the minimum extent necessary to +make such provision valid and enforceable. +No term or provision of this License shall be deemed waived and no breach +consented to unless such waiver or consent shall be in writing and signed by the +party to be charged with such waiver or consent. +This License constitutes the entire agreement between the parties with respect +to the Work licensed here. There are no understandings, agreements or +representations with respect to the Work not specified here. Licensor shall not +be bound by any additional provisions that may appear in any communication from +You. This License may not be modified without the mutual written agreement of +the Licensor and You. + +Apache Commons Collections +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +Apache Commons Compress +Copyright 2002-2012 The Apache Software Foundation + +flink-hadoop-fs +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +The binary distribution of this product bundles binaries of +org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the +following notices: +* Copyright 2011 Dain Sundstrom +* Copyright 2011 FuseSource Corp. http://fusesource.com + +The binary distribution of this product bundles binaries of +org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni), +which has the following notices: +* This product includes software developed by FuseSource Corp. + http://fusesource.com +* This product includes software developed at + Progress Software Corporation and/or its subsidiaries or affiliates. +* This product includes software developed by IBM Corporation and others. + +The binary distribution of this product bundles binaries of +AWS Java SDK 1.10.6, +which has the following notices: + * This software includes third party software subject to the following + copyrights: - XML parsing and utility functions from JetS3t - Copyright + 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org - + Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility + functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. + +The binary distribution of this product bundles binaries of +Gson 2.2.4, +which has the following notices: + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2014 The Netty Project + +The Netty Project 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. + +Also, please refer to each LICENSE..txt file, which is located in +the 'license' directory of the distribution file, for the license terms of the +components that this product depends on. + +------------------------------------------------------------------------------- +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified portion of 'Webbit', an event based +WebSocket and HTTP server, which can be obtained at: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +This product contains a modified portion of 'SLF4J', a simple logging +facade for Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +This product contains a modified portion of 'ArrayDeque', written by Josh +Bloch of Google, Inc: + + * LICENSE: + * license/LICENSE.deque.txt (Public Domain) + +This product contains a modified portion of 'Apache Harmony', an open source +Java SE, which can be obtained at: + + * LICENSE: + * license/LICENSE.harmony.txt (Apache License 2.0) + * HOMEPAGE: + * http://archive.apache.org/dist/harmony/ + +This product contains a modified version of Roland Kuhn's ASL2 +AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue. +It can be obtained at: + + * LICENSE: + * license/LICENSE.abstractnodequeue.txt (Public Domain) + * HOMEPAGE: + * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java + +This product contains a modified portion of 'jbzip2', a Java bzip2 compression +and decompression library written by Matthew J. Francis. It can be obtained at: + + * LICENSE: + * license/LICENSE.jbzip2.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jbzip2/ + +This product contains a modified portion of 'libdivsufsort', a C API library to construct +the suffix array and the Burrows-Wheeler transformed string for any input string of +a constant-size alphabet written by Yuta Mori. It can be obtained at: + + * LICENSE: + * license/LICENSE.libdivsufsort.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/libdivsufsort/ + +This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM, + which can be obtained at: + + * LICENSE: + * license/LICENSE.jctools.txt (ASL2 License) + * HOMEPAGE: + * https://github.com/JCTools/JCTools + +This product optionally depends on 'JZlib', a re-implementation of zlib in +pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product optionally depends on 'Compress-LZF', a Java library for encoding and +decoding data in LZF format, written by Tatu Saloranta. It can be obtained at: + + * LICENSE: + * license/LICENSE.compress-lzf.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/ning/compress + +This product optionally depends on 'lz4', a LZ4 Java compression +and decompression library written by Adrien Grand. It can be obtained at: + + * LICENSE: + * license/LICENSE.lz4.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jpountz/lz4-java + +This product optionally depends on 'lzma-java', a LZMA Java compression +and decompression library, which can be obtained at: + + * LICENSE: + * license/LICENSE.lzma-java.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jponge/lzma-java + +This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression +and decompression library written by William Kinney. It can be obtained at: + + * LICENSE: + * license/LICENSE.jfastlz.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jfastlz/ + +This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/protobuf/ + +This product optionally depends on 'Bouncy Castle Crypto APIs' to generate +a temporary self-signed X.509 certificate when the JVM does not provide the +equivalent functionality. It can be obtained at: + + * LICENSE: + * license/LICENSE.bouncycastle.txt (MIT License) + * HOMEPAGE: + * http://www.bouncycastle.org/ + +This product optionally depends on 'Snappy', a compression library produced +by Google Inc, which can be obtained at: + + * LICENSE: + * license/LICENSE.snappy.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/snappy/ + +This product optionally depends on 'JBoss Marshalling', an alternative Java +serialization API, which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://www.jboss.org/jbossmarshalling + +This product optionally depends on 'Caliper', Google's micro- +benchmarking framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.caliper.txt (Apache License 2.0) + * HOMEPAGE: + * http://code.google.com/p/caliper/ + +This product optionally depends on 'Apache Commons Logging', a logging +framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + +This product optionally depends on 'Apache Log4J', a logging framework, which +can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + +This product optionally depends on 'Aalto XML', an ultra-high performance +non-blocking XML processor, which can be obtained at: + + * LICENSE: + * license/LICENSE.aalto-xml.txt (Apache License 2.0) + * HOMEPAGE: + * http://wiki.fasterxml.com/AaltoHome + +This product contains a modified version of 'HPACK', a Java implementation of +the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at: + + * LICENSE: + * license/LICENSE.hpack.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/twitter/hpack + +This product contains a modified portion of 'Apache Commons Lang', a Java library +provides utilities for the java.lang API, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-lang.txt (Apache License 2.0) + * HOMEPAGE: + * https://commons.apache.org/proper/commons-lang/ + +The binary distribution of this product bundles binaries of +Commons Codec 1.4, +which has the following notices: + * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + =============================================================================== + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +The binary distribution of this product bundles binaries of +Commons Lang 2.6, +which has the following notices: + * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +The binary distribution of this product bundles binaries of +Apache Log4j 1.2.17, +which has the following notices: + * ResolverUtil.java + Copyright 2005-2006 Tim Fennell + Dumbster SMTP test server + Copyright 2004 Jason Paul Kitchen + TypeUtil.java + Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams + +The binary distribution of this product bundles binaries of +Java Concurrency in Practice book annotations 1.0, +which has the following notices: + * Copyright (c) 2005 Brian Goetz and Tim Peierls Released under the Creative + Commons Attribution License (http://creativecommons.org/licenses/by/2.5) + Official home: http://www.jcip.net Any republication or derived work + distributed in source code form must include this copyright and license + notice. + +The binary distribution of this product bundles binaries of +Jetty 6.1.26, +which has the following notices: + * ============================================================== + Jetty Web Container + Copyright 1995-2016 Mort Bay Consulting Pty Ltd. + ============================================================== + + The Jetty Web Container is Copyright Mort Bay Consulting Pty Ltd + unless otherwise noted. + + Jetty is dual licensed under both + + * The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html + + and + + * The Eclipse Public 1.0 License + http://www.eclipse.org/legal/epl-v10.html + + Jetty may be distributed under either license. + + ------ + Eclipse + + The following artifacts are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + + The following artifacts are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + + The following artifacts are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + + ------ + Oracle + + The following artifacts are CDDL + GPLv2 with classpath exception. + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * javax.servlet:javax.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + + ------ + Oracle OpenJDK + + If ALPN is used to negotiate HTTP/2 connections, then the following + artifacts may be included in the distribution or downloaded when ALPN + module is selected. + + * java.sun.security.ssl + + These artifacts replace/modify OpenJDK classes. The modififications + are hosted at github and both modified and original are under GPL v2 with + classpath exceptions. + http://openjdk.java.net/legal/gplv2+ce.html + + ------ + OW2 + + The following artifacts are licensed by the OW2 Foundation according to the + terms of http://asm.ow2.org/license.html + + org.ow2.asm:asm-commons + org.ow2.asm:asm + + ------ + Apache + + The following artifacts are ASL2 licensed. + + org.apache.taglibs:taglibs-standard-spec + org.apache.taglibs:taglibs-standard-impl + + ------ + MortBay + + The following artifacts are ASL2 licensed. Based on selected classes from + following Apache Tomcat jars, all ASL2 licensed. + + org.mortbay.jasper:apache-jsp + org.apache.tomcat:tomcat-jasper + org.apache.tomcat:tomcat-juli + org.apache.tomcat:tomcat-jsp-api + org.apache.tomcat:tomcat-el-api + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-api + org.apache.tomcat:tomcat-util-scan + org.apache.tomcat:tomcat-util + + org.mortbay.jasper:apache-el + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-el-api + + ------ + Mortbay + + The following artifacts are CDDL + GPLv2 with classpath exception. + + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + org.eclipse.jetty.toolchain:jetty-schemas + + ------ + Assorted + + The UnixCrypt.java code implements the one way cryptography used by + Unix systems for simple password protection. Copyright 1996 Aki Yoshida, + modified April 2001 by Iris Van den Broeke, Daniel Deville. + Permission to use, copy, modify and distribute UnixCrypt + for non-commercial or commercial purposes and without fee is + granted provided that the copyright notice appears in all copies./ + +The binary distribution of this product bundles binaries of +Snappy for Java 1.0.4.1, +which has the following notices: + * This product includes software developed by Google + Snappy: http://code.google.com/p/snappy/ (New BSD License) + + This product includes software developed by Apache + PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ + (Apache 2.0 license) + + This library containd statically linked libstdc++. This inclusion is allowed by + "GCC RUntime Library Exception" + http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html + + == Contributors == + * Tatu Saloranta + * Providing benchmark suite + * Alec Wysoker + * Performance and memory usage improvement + +The binary distribution of this product bundles binaries of +Xerces2 Java Parser 2.9.1, +which has the following notices: + * ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the Apache Xerces Java distribution. == + ========================================================================= + + Apache Xerces Java + Copyright 1999-2007 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of this software were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - voluntary contributions made by Paul Eng on behalf of the + Apache Software Foundation that were originally developed at iClick, Inc., + software copyright (c) 1999. + +Apache Commons CLI +Copyright 2001-2015 The Apache Software Foundation + +Apache Commons Math +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed for Orekit by +CS Systèmes d'Information (http://www.c-s.fr/) +Copyright 2010-2012 CS Systèmes d'Information + +Apache HttpClient +Copyright 1999-2017 The Apache Software Foundation + +Apache HttpCore +Copyright 2005-2017 The Apache Software Foundation + +Apache Commons Codec +Copyright 2002-2014 The Apache Software Foundation + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons Net +Copyright 2001-2012 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Configuration +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Digester +Copyright 2001-2008 The Apache Software Foundation + +Apache Commons BeanUtils +Copyright 2000-2010 The Apache Software Foundation + +htrace-core4 +Copyright 2015 The Apache Software Foundation + +This product currently only contains code developed by authors +of specific components, as identified by the source code files; +if such notes are missing files have been created by +Tatu Saloranta. + +For additional credits (generally to people who reported problems) +see CREDITS file. + +Apache Jakarta HttpClient +Copyright 1999-2007 The Apache Software Foundation + +Apache Commons Logging +Copyright 2003-2013 The Apache Software Foundation + +flink-shaded-hadoop2-uber +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.jamesmurty.utils:java-xmlbuilder:0.4 +- commons-beanutils:commons-beanutils:1.9.3 +- commons-cli:commons-cli:1.3.1 +- commons-codec:commons-codec:1.10 +- commons-collections:commons-collections:3.2.2 +- commons-configuration:commons-configuration:1.7 +- commons-daemon:commons-daemon:1.0.13 +- commons-digester:commons-digester:1.8.1 +- commons-io:commons-io:2.4 +- commons-lang:commons-lang:2.6 +- commons-logging:commons-logging:1.1.3 +- commons-net:commons-net:3.1 +- org.apache.avro:avro:1.8.2 +- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-math3:3.5 +- org.apache.zookeeper:zookeeper:3.4.10 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 +- org.xerial.snappy:snappy-java:1.1.4 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.jcraft:jsch:0.1.42 +- com.thoughtworks.paranamer:paranamer:2.7 +- xmlenc:xmlenc:0.52 + +This project bundles the following dependencies under the Common Development and Distribution License (CDDL) 1.0. +See bundled license files for details. + +- javax.activation:activation:1.1 (http://java.sun.com/products/javabeans/jaf/index.jsp) +- javax.servlet:servlet-api:2.5 (https://repo1.maven.org/maven2/javax/servlet/servlet-api/2.5/) +- javax.xml.stream:stax-api:1.0-2 (http://central.maven.org/maven2/javax/xml/stream/stax-api/1.0-2/) + +This project bundles the following dependencies under the Common Development and Distribution License (CDDL) 1.1. +See bundled license files for details. + +- com.sun.jersey:jersey-client:1.9 (http://central.maven.org/maven2/com/sun/jersey/jersey-client/1.9/) +- javax.xml.bind:jaxb-api:2.2.2 (https://jaxb.dev.java.net/) + +This project bundles "org.tukaani:xz:1.5". +This Java implementation of XZ has been put into the public domain, thus you can do +whatever you want with it. All the files in the package have been written by Lasse Collin, +but some files are heavily based on public domain code written by Igor Pavlov. + + +flink-shaded-hadoop2 +Copyright 2014-2018 The Apache Software Foundation + +- com.google.guava:guava:11.0.2 +- net.java.dev.jets3t:jets3t:0.9.0 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- commons-httpclient:commons-httpclient:3.1 +- org.codehaus.jackson:jackson-core-asl:1.9.13 +- org.codehaus.jackson:jackson-mapper-asl:1.9.13 + +The version of the actual Hadoop dependencies depend on the built. + +- org.apache.hadoop:hadoop-common +- org.apache.hadoop:hadoop-hdfs +- org.apache.hadoop:hadoop-mapreduce-client-core +- org.apache.hadoop:hadoop-yarn-client +- org.apache.hadoop:hadoop-yarn-common +- org.apache.hadoop:hadoop-annotations +- org.apache.hadoop:hadoop-auth +- org.apache.hadoop:hadoop-yarn-api + +- com.google.protobuf.protobuf-java:2.5.0 + +This product currently only contains code developed by authors +of specific components, as identified by the source code files; +if such notes are missing files have been created by +Tatu Saloranta. + +For additional credits (generally to people who reported problems) +see CREDITS file. + +Apache Jakarta HttpClient +Copyright 1999-2007 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +Apache HttpClient +Copyright 1999-2017 The Apache Software Foundation + +Apache HttpCore +Copyright 2005-2017 The Apache Software Foundation + +Apache Commons Compress +Copyright 2002-2012 The Apache Software Foundation + +Apache Avro +Copyright 2009-2017 The Apache Software Foundation + +Apache Commons CLI +Copyright 2001-2015 The Apache Software Foundation + +Apache Commons Math +Copyright 2001-2015 The Apache Software Foundation + +This product includes software developed for Orekit by +CS Systèmes d'Information (http://www.c-s.fr/) +Copyright 2010-2012 CS Systèmes d'Information + +Apache Commons Codec +Copyright 2002-2014 The Apache Software Foundation + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons Net +Copyright 2001-2012 The Apache Software Foundation + +Apache Commons Collections +Copyright 2001-2015 The Apache Software Foundation + +Apache Commons Logging +Copyright 2003-2013 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Configuration +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Digester +Copyright 2001-2008 The Apache Software Foundation + +Apache Commons BeanUtils +Copyright 2000-2016 The Apache Software Foundation + +Apache Commons Daemon +Copyright 1999-2013 The Apache Software Foundation + +flink-s3-fs-presto +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.facebook.presto:presto-hive:0.185 +- com.facebook.presto.hadoop:hadoop-apache2:2.7.3-1 +- com.google.guava:guava:21.0 +- io.airlift:configuration:0.148 +- io.airlift:log:0.148 +- io.airlift:stats:0.148 +- io.airlift:units:1.0 +- io.airlift:slice:0.31 +- com.fasterxml.jackson.core:jackson-annotations:2.8.1 +- com.fasterxml.jackson.core:jackson-core:2.8.1 +- com.fasterxml.jackson.core:jackson-databind:2.8.1 +- joda-time:joda-time:2.5 +- org.weakref:jmxutils:1.19 + +This project bundles the following dependencies under the Creative Commons CC0 1.0 Universal Public Domain Dedication License (http://creativecommons.org/publicdomain/zero/1.0/) +See bundled license files for details. + +- org.hdrhistogram:HdrHistogram:2.1.9 + + +flink-s3-fs-base +Copyright 2014-2018 The Apache Software Foundation + +- org.apache.hadoop:hadoop-aws:3.1.0 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpclient:4.5.3 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- com.amazonaws:aws-java-sdk-core:1.11.271 +- com.amazonaws:aws-java-sdk-dynamodb:1.11.271 +- com.amazonaws:aws-java-sdk-kms:1.11.271 +- com.amazonaws:aws-java-sdk-s3:1.11.271 +- com.amazonaws:jmespath-java:1.11.271 +- software.amazon.ion:ion-java:1.0.2 +- com.fasterxml.jackson.core:jackson-annotations:2.6.0 +- com.fasterxml.jackson.core:jackson-core:2.6.7 +- com.fasterxml.jackson.core:jackson-databind:2.6.7.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.6.7 +- joda-time:joda-time:2.5 + +flink-fs-hadoop-shaded +Copyright 2014-2018 The Apache Software Foundation + +- org.apache.hadoop:hadoop-annotations:3.1.0 +- org.apache.hadoop:hadoop-auth:3.1.0 +- org.apache.hadoop:hadoop-common:3.1.0 +- org.apache.htrace:htrace-core4:4.1.0-incubating +- org.apache.commons:commons-configuration2:2.1.1 +- org.apache.commons:commons-lang3:3.3.2 +- commons-lang:commons-lang:2.6 +- commons-collections:commons-collections:3.2.2 +- commons-io:commons-io:2.4 +- commons-logging:commons-logging:1.1.3 +- commons-beanutils:commons-beanutils:1.9.3 +- com.google.guava:guava:11.0.2 +- com.fasterxml.jackson.core:jackson-annotations:2.7.0 +- com.fasterxml.jackson.core:jackson-core:2.7.8 +- com.fasterxml.jackson.core:jackson-databind:2.7.8 +- com.fasterxml.woodstox:woodstox-core:5.0.3 + +This project bundles the following dependencies under the Go License (https://golang.org/LICENSE). +See bundled license files for details. + +- com.google.re2j:re2j:1.1 + +This project bundles the following dependencies under BSD License (https://opensource.org/licenses/bsd-license.php). +See bundled license files for details. + +- org.codehaus.woodstox:stax2-api:3.1.4 (https://github.com/FasterXML/stax2-api/tree/stax2-api-3.1.4) + +This project bundles org.apache.hadoop:*:3.1.0 from which it inherits the following notices: + +The Apache Hadoop project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + +For the org.apache.hadoop.util.bloom.* classes: + +/** + * + * Copyright (c) 2005, European Commission project OneLab under contract + * 034819 (http://www.one-lab.org) + * All rights reserved. + * Redistribution and use in source and binary forms, with or + * without modification, are permitted provided that the following + * conditions are met: + * - Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * - Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the distribution. + * - Neither the name of the University Catholique de Louvain - UCL + * nor the names of its contributors may be used to endorse or + * promote products derived from this software without specific prior + * written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, + * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN + * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE + * POSSIBILITY OF SUCH DAMAGE. + */ + +For portions of the native implementation of slicing-by-8 CRC calculation +in src/main/native/src/org/apache/hadoop/util: + +Copyright (c) 2008,2009,2010 Massachusetts Institute of Technology. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. +* Neither the name of the Massachusetts Institute of Technology nor + the names of its contributors may be used to endorse or promote + products derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Other portions are under the same license from Intel: +http://sourceforge.net/projects/slicing-by-8/ +/*++ + * + * Copyright (c) 2004-2006 Intel Corporation - All Rights Reserved + * + * This software program is licensed subject to the BSD License, + * available at http://www.opensource.org/licenses/bsd-license.html + * + * Abstract: The main routine + * + --*/ + +For src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,lz4hc.c}, + +/* + LZ4 - Fast LZ compression algorithm + Header File + Copyright (C) 2011-2014, Yann Collet. + BSD 2-Clause License (http://www.opensource.org/licenses/bsd-license.php) + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + You can contact the author at : + - LZ4 source repository : http://code.google.com/p/lz4/ + - LZ4 public forum : https://groups.google.com/forum/#!forum/lz4c +*/ + +For hadoop-common-project/hadoop-common/src/main/native/gtest +--------------------------------------------------------------------- +Copyright 2008, Google Inc. +All rights reserved. + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +The binary distribution of this product bundles these dependencies under the +following license: +re2j 1.1 +--------------------------------------------------------------------- +(GO license) +This is a work derived from Russ Cox's RE2 in Go, whose license +http://golang.org/LICENSE is as follows: + +Copyright (c) 2009 The Go Authors. All rights reserved. + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +For hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/util/tree.h +--------------------------------------------------------------------- +Copyright 2002 Niels Provos +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles binaries of leveldbjni +(https://github.com/fusesource/leveldbjni), which is available under the +following license: + +Copyright (c) 2011 FuseSource Corp. All rights reserved. + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of FuseSource Corp. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +For hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/compat/{fstatat|openat|unlinkat}.h: + +Copyright (c) 2012 The FreeBSD Foundation +All rights reserved. + +This software was developed by Pawel Jakub Dawidek under sponsorship from +the FreeBSD Foundation. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE AUTHORS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHORS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS +OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + +============= + +The binary distribution of this product bundles binaries of leveldb +(http://code.google.com/p/leveldb/), which is available under the following +license: + +Copyright (c) 2011 The LevelDB Authors. All rights reserved. + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +The binary distribution of this product bundles binaries of snappy +(http://code.google.com/p/snappy/), which is available under the following +license: + +Copyright 2011, Google Inc. +All rights reserved. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.9.4/ +-------------------------------------------------------------------------------- +Copyright (C) 2008-2016, SpryMedia Ltd. + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2010 Aleksander Williams + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js +-------------------------------------------------------------------------------- + +Copyright (c) 2011-2016 Tim Wood, Iskren Chernev, Moment.js contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. + +The binary distribution of this product bundles these dependencies under the +following license: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.0.2 +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css +bootstrap v3.3.6 +broccoli-asset-rev v2.4.2 +broccoli-funnel v1.0.1 +datatables v1.10.8 +em-helpers v0.5.13 +em-table v0.1.6 +ember v2.2.0 +ember-array-contains-helper v1.0.2 +ember-bootstrap v0.5.1 +ember-cli v1.13.13 +ember-cli-app-version v1.0.0 +ember-cli-babel v5.1.6 +ember-cli-content-security-policy v0.4.0 +ember-cli-dependency-checker v1.2.0 +ember-cli-htmlbars v1.0.2 +ember-cli-htmlbars-inline-precompile v0.3.1 +ember-cli-ic-ajax v0.2.1 +ember-cli-inject-live-reload v1.4.0 +ember-cli-jquery-ui v0.0.20 +ember-cli-qunit v1.2.1 +ember-cli-release v0.2.8 +ember-cli-shims v0.0.6 +ember-cli-sri v1.2.1 +ember-cli-test-loader v0.2.1 +ember-cli-uglify v1.2.0 +ember-d3 v0.1.0 +ember-data v2.1.0 +ember-disable-proxy-controllers v1.0.1 +ember-export-application-global v1.0.5 +ember-load-initializers v0.1.7 +ember-qunit v0.4.16 +ember-qunit-notifications v0.1.0 +ember-resolver v2.0.3 +ember-spin-spinner v0.2.3 +ember-truth-helpers v1.2.0 +jquery v2.1.4 +jquery-ui v1.11.4 +loader.js v3.3.0 +momentjs v2.10.6 +qunit v1.19.0 +select2 v4.0.0 +snippet-ss v1.11.0 +spin.js v2.3.2 +Azure Data Lake Store - Java client SDK 2.0.11 +JCodings 1.0.8 +Joni 2.1.2 +Mockito 1.8.5 +JUL to SLF4J bridge 1.7.25 +SLF4J API Module 1.7.25 +SLF4J LOG4J-12 Binding 1.7.25 +-------------------------------------------------------------------------------- + +The MIT License (MIT) + +For: +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery +Apache HBase - Server which contains JQuery minified javascript library version 1.8.3 +Microsoft JDBC Driver for SQLServer - version 6.2.1.jre7 +-------------------------------------------------------------------------------- + +MIT License + +Copyright (c) 2003-2017 Optimatika + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +For: +oj! Algorithms - version 43.0 +-------------------------------------------------------------------------------- + +Copyright 2005, 2012, 2013 jQuery Foundation and other contributors, https://jquery.org/ + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/jquery/jquery + +The following license applies to all parts of this software except as +documented below: + +==== + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +All files located in the node_modules and external directories are +externally maintained libraries used by this software which have their +own licenses; we recommend you read them, as their terms may differ from +the terms above. + +For: +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js +-------------------------------------------------------------------------------- + +Copyright (c) 2014 Ivan Bozhanov + +For: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js +-------------------------------------------------------------------------------- + +D3 is available under a 3-clause BSD license. For details, see: +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3-LICENSE + +The binary distribution of this product bundles these dependencies under the +following license: +HSQLDB Database 2.3.4 +-------------------------------------------------------------------------------- +(HSQL License) +"COPYRIGHTS AND LICENSES (based on BSD License) + +For work developed by the HSQL Development Group: + +Copyright (c) 2001-2016, The HSQL Development Group +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this +list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, +this list of conditions and the following disclaimer in the documentation +and/or other materials provided with the distribution. + +Neither the name of the HSQL Development Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL HSQL DEVELOPMENT GROUP, HSQLDB.ORG, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For work originally developed by the Hypersonic SQL Group: + +Copyright (c) 1995-2000 by the Hypersonic SQL Group. +All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +Neither the name of the Hypersonic SQL Group nor the names of its +contributors may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ""AS IS"" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE HYPERSONIC SQL GROUP, +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +This software consists of voluntary contributions made by many individuals on behalf of the +Hypersonic SQL Group." + +The binary distribution of this product bundles these dependencies under the +following license: +Java Servlet API 3.1.0 +servlet-api 2.5 +jsp-api 2.1 +jsr311-api 1.1.1 +Glassfish Jasper 6.1.14 +Servlet Specification 2.5 API 6.1.14 +-------------------------------------------------------------------------------- +(CDDL 1.0) +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 +1. Definitions.  + +1.1. Contributor means each individual or entity +that creates or contributes to the creation of +Modifications.  + +1.2. Contributor Version means the combination of the +Original Software, prior Modifications used by a Contributor (if any), and the +Modifications made by that particular Contributor.  + +1.3. Covered +Software means (a) the Original Software, or (b) Modifications, or (c) the +combination of files containing Original Software with files containing +Modifications, in each case including portions +thereof.  + +1.4. Executable means the Covered Software in any form other +than Source Code.  + +1.5. Initial Developer means the individual or entity +that first makes Original Software available under this +License.  + +1.6. Larger Work means a work which combines Covered Software or +portions thereof with code not governed by the terms of this +License.  + +1.7. License means this document.  + +1.8. Licensable means +having the right to grant, to the maximum extent possible, whether at the time +of the initial grant or subsequently acquired, any and all of the rights +conveyed herein.  + +1.9. Modifications means the Source Code and Executable +form of any of the following: +A. Any file that results from an addition to, +deletion from or modification of the contents of a file containing Original +Software or previous Modifications; +B. Any new file that contains any part of the Original Software +or previous Modification; or +C. Any new file that is contributed or otherwise made available +under the terms of this License.  + +1.10. Original Software means the Source Code and Executable form of +computer software code that is originally released under this License.  + +1.11. Patent Claims means any patent claim(s), now owned or +hereafter acquired, including without limitation, method, process, and apparatus +claims, in any patent Licensable by grantor.  + +1.12. Source Code means (a) the common form of computer software code in which +modifications are made and (b) associated documentation included in or +with such code.  + +1.13. You (or Your) means an individual or a legal entity exercising rights +under, and complying with all of the terms of, this License. For legal entities, +You includes any entity which controls, is controlled by, or is under common control +with You. For purposes of this definition, control means (a) the power, direct +or indirect, to cause the direction or management of such entity, whether by +contract or otherwise, or (b) ownership of more than fifty percent (50%) of the +outstanding shares or beneficial ownership of such entity.  + +2. License Grants. + +2.1. The Initial Developer Grant. Conditioned upon Your compliance +with Section 3.1 below and subject to third party intellectual property claims, +the Initial Developer hereby grants You a world-wide, royalty-free, +non-exclusive license:  + +(a) under intellectual property rights (other than +patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, +display, perform, sublicense and distribute the Original Software (or portions +thereof), with or without Modifications, and/or as part of a Larger Work; +and  + +(b) under Patent Claims infringed by the making, using or selling of +Original Software, to make, have made, use, practice, sell, and offer for sale, +and/or otherwise dispose of the Original Software (or portions +thereof); + +(c) The licenses granted in Sections 2.1(a) and (b) are +effective on the date Initial Developer first distributes or otherwise makes the +Original Software available to a third party under the terms of this +License; + +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or +(ii) the combination of the Original Software with other software or +devices.  + +2.2. Contributor Grant. Conditioned upon Your compliance with +Section 3.1 below and subject to third party intellectual property claims, each +Contributor hereby grants You a world-wide, royalty-free, non-exclusive +license:  + +(a) under intellectual property rights (other than patent or +trademark) Licensable by Contributor to use, reproduce, modify, display, +perform, sublicense and distribute the Modifications created by such Contributor +(or portions thereof), either on an unmodified basis, with other Modifications, +as Covered Software and/or as part of a Larger Work; and  + +(b) under Patent +Claims infringed by the making, using, or selling of Modifications made by that +Contributor either alone and/or in combination with its Contributor Version (or +portions of such combination), to make, use, sell, offer for sale, have made, +and/or otherwise dispose of: (1) Modifications made by that Contributor (or +portions thereof); and (2) the combination of Modifications made by that +Contributor with its Contributor Version (or portions of such +combination).  + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are +effective on the date Contributor first distributes or otherwise makes the +Modifications available to a third party. + +(d) Notwithstanding Section 2.2(b) +above, no patent license is granted: (1) for any code that Contributor has +deleted from the Contributor Version; (2) for infringements caused by: +(i) third party modifications of Contributor Version, or (ii) the combination +of Modifications made by that Contributor with other software (except as part of +the Contributor Version) or other devices; or (3) under Patent Claims infringed +by Covered Software in the absence of Modifications made by that +Contributor.  + +3. Distribution Obligations.  + +3.1. Availability of Source +Code. Any Covered Software that You distribute or otherwise make available in +Executable form must also be made available in Source Code form and that Source +Code form must be distributed only under the terms of this License. You must +include a copy of this License with every copy of the Source Code form of the +Covered Software You distribute or otherwise make available. You must inform +recipients of any such Covered Software in Executable form as to how they can +obtain such Covered Software in Source Code form in a reasonable manner on or +through a medium customarily used for software exchange.  + +3.2. +Modifications. The Modifications that You create or to which You contribute are +governed by the terms of this License. You represent that You believe Your +Modifications are Your original creation(s) and/or You have sufficient rights to +grant the rights conveyed by this License.  + +3.3. Required Notices. You must +include a notice in each of Your Modifications that identifies You as the +Contributor of the Modification. You may not remove or alter any copyright, +patent or trademark notices contained within the Covered Software, or any +notices of licensing or any descriptive text giving attribution to any +Contributor or the Initial Developer.  + +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source Code +form that alters or restricts the applicable version of this License or the +recipients rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. + +3.5. Distribution of Executable Versions. You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipients rights in the Source Code form from the rights set forth in this +License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer.  + +3.6. Larger Works. You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software.  + +4. Versions of the License.  + +4.1. +New Versions. Sun Microsystems, Inc. is the initial license steward and may +publish revised and/or new versions of this License from time to time. Each +version will be given a distinguishing version number. Except as provided in +Section 4.3, no one other than the license steward has the right to modify this +License.  + +4.2. Effect of New Versions. You may always continue to use, +distribute or otherwise make the Covered Software available under the terms of +the version of the License under which You originally received the Covered +Software. If the Initial Developer includes a notice in the Original Software +prohibiting it from being distributed or otherwise made available under any +subsequent version of the License, You must distribute and make the Covered +Software available under the terms of the version of the License under which You +originally received the Covered Software. Otherwise, You may also choose to use, +distribute or otherwise make the Covered Software available under the terms of +any subsequent version of the License published by the license +steward.  + +4.3. Modified Versions. When You are an Initial Developer and You +want to create a new license for Your Original Software, You may create and use +a modified version of this License if You: (a) rename the license and remove +any references to the name of the license steward (except to note that the +license differs from this License); and (b) otherwise make it clear that the +license contains terms which differ from this License.  + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, +WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT +LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, +MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY +COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER +OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR +CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS +LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER +THIS DISCLAIMER.  + +6. TERMINATION.  + +6.1. This License and the rights +granted hereunder will terminate automatically if You fail to comply with terms +herein and fail to cure such breach within 30 days of becoming aware of the +breach. Provisions which, by their nature, must remain in effect beyond the +termination of this License shall survive.  + +6.2. If You assert a patent +infringement claim (excluding declaratory judgment actions) against Initial +Developer or a Contributor (the Initial Developer or Contributor against whom +You assert such claim is referred to as Participant) alleging that the +Participant Software (meaning the Contributor Version where the Participant is a +Contributor or the Original Software where the Participant is the Initial +Developer) directly or indirectly infringes any patent, then any and all rights +granted directly or indirectly to You by such Participant, the Initial Developer +(if the Initial Developer is not the Participant) and all Contributors under +Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from +Participant terminate prospectively and automatically at the expiration of such +60 day notice period, unless if within such 60 day period You withdraw Your +claim with respect to the Participant Software against such Participant either +unilaterally or pursuant to a written agreement with Participant.  + +6.3. In +the event of termination under Sections 6.1 or 6.2 above, all end user licenses +that have been validly granted by You or any distributor hereunder prior to +termination (excluding licenses granted to You by any distributor) shall survive +termination.  + +7. LIMITATION OF LIABILITY. +UNDER NO CIRCUMSTANCES AND UNDER +NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, +SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF +COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY +PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY +CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF +GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER +COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE +POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO +LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO +THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT +ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO +THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.  + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a commercial item, as that term is defined in +48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as +that term is defined at 48 C.F.R.  252.227-7014(a)(1)) and commercial computer +software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. +1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through +227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software +with only those rights set forth herein. This U.S. Government Rights clause is +in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision +that addresses Government rights in computer software under this +License.  + +9. MISCELLANEOUS. +This License represents the complete agreement +concerning subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdictions conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software.  + +10. RESPONSIBILITY FOR CLAIMS. +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability.  + +The binary distribution of this product bundles these dependencies under the +following license: +jersey-client 1.19 +jersey-core 1.19 +jersey-grizzly2 1.19 +jersey-grizzly2-servlet 1.19 +jersey-json 1.19 +jersey-server 1.19 +jersey-servlet 1.19 +jersey-guice 1.19 +Jersey Test Framework - Grizzly 2 Module 1.19 +JAXB RI 2.2.3 +Java Architecture for XML Binding 2.2.11 +grizzly-framework 2.2.21 +grizzly-http 2.2.21 +grizzly-http-server 2.2.21 +grizzly-http-servlet 2.2.21 +grizzly-rcm 2.2.21 +-------------------------------------------------------------------------------- +(CDDL 1.1) +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 + +1. Definitions. + +1.1. “Contributor” means each individual or entity that creates or +contributes to the creation of Modifications. +1.2. “Contributor Version” means the combination of the Original Software, +prior Modifications used by a Contributor (if any), and the Modifications made +by that particular Contributor. +1.3. “Covered Software” means (a) the Original Software, or (b) +Modifications, or (c) the combination of files containing Original Software with +files containing Modifications, in each case including portions thereof. +1.4. “Executable” means the Covered Software in any form other than Source +Code. +1.5. “Initial Developer” means the individual or entity that first makes +Original Software available under this License. +1.6. “Larger Work” means a work which combines Covered Software or portions +thereof with code not governed by the terms of this License. +1.7. “License” means this document. +1.8. “Licensable” means having the right to grant, to the maximum extent +possible, whether at the time of the initial grant or subsequently acquired, any +and all of the rights conveyed herein. +1.9. “Modifications” means the Source Code and Executable form of any of the +following: +A. Any file that results from an addition to, deletion from or modification of +the contents of a file containing Original Software or previous Modifications; +B. Any new file that contains any part of the Original Software or previous +Modification; or +C. Any new file that is contributed or otherwise made available under the terms +of this License. +1.10. “Original Software” means the Source Code and Executable form of +computer software code that is originally released under this License. +1.11. “Patent Claims” means any patent claim(s), now owned or hereafter +acquired, including without limitation, method, process, and apparatus claims, +in any patent Licensable by grantor. +1.12. “Source Code” means (a) the common form of computer software code in +which modifications are made and (b) associated documentation included in or +with such code. +1.13. “You” (or “Your”) means an individual or a legal entity exercising +rights under, and complying with all of the terms of, this License. For legal +entities, “You” includes any entity which controls, is controlled by, or is +under common control with You. For purposes of this definition, “control” +means (a) the power, direct or indirect, to cause the direction or management of +such entity, whether by contract or otherwise, or (b) ownership of more than +fifty percent (50%) of the outstanding shares or beneficial ownership of such +entity. + +2.1. The Initial Developer Grant. + +Conditioned upon Your compliance with Section 3.1 below and subject to +third party intellectual property claims, the Initial Developer hereby grants +You a world-wide, royalty-free, non-exclusive license: +(a) under intellectual +property rights (other than patent or trademark) Licensable by Initial +Developer, to use, reproduce, modify, display, perform, sublicense and +distribute the Original Software (or portions thereof), with or without +Modifications, and/or as part of a Larger Work; and +(b) under Patent Claims +infringed by the making, using or selling of Original Software, to make, have +made, use, practice, sell, and offer for sale, and/or otherwise dispose of the +Original Software (or portions thereof). +(c) The licenses granted in Sections +2.1(a) and (b) are effective on the date Initial Developer first distributes or +otherwise makes the Original Software available to a third party under the terms +of this License. +(d) Notwithstanding Section 2.1(b) above, no patent license is +granted: (1) for code that You delete from the Original Software, or (2) for +infringements caused by: (i) the modification of the Original Software, or (ii) +the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. + +Conditioned upon Your compliance with Section 3.1 below and +subject to third party intellectual property claims, each Contributor hereby +grants You a world-wide, royalty-free, non-exclusive license: +(a) under +intellectual property rights (other than patent or trademark) Licensable by +Contributor to use, reproduce, modify, display, perform, sublicense and +distribute the Modifications created by such Contributor (or portions thereof), +either on an unmodified basis, with other Modifications, as Covered Software +and/or as part of a Larger Work; and +(b) under Patent Claims infringed by the +making, using, or selling of Modifications made by that Contributor either alone +and/or in combination with its Contributor Version (or portions of such +combination), to make, use, sell, offer for sale, have made, and/or otherwise +dispose of: (1) Modifications made by that Contributor (or portions thereof); +and (2) the combination of Modifications made by that Contributor with its +Contributor Version (or portions of such combination). +(c) The licenses granted +in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first +distributes or otherwise makes the Modifications available to a third +party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: +(1) for any code that Contributor has deleted from the Contributor Version; (2) +for infringements caused by: (i) third party modifications of Contributor +Version, or (ii) the combination of Modifications made by that Contributor with +other software (except as part of the Contributor Version) or other devices; or +(3) under Patent Claims infringed by Covered Software in the absence of +Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. +Any Covered Software that You distribute or +otherwise make available in Executable form must also be made available in +Source Code form and that Source Code form must be distributed only under the +terms of this License. You must include a copy of this License with every copy +of the Source Code form of the Covered Software You distribute or otherwise make +available. You must inform recipients of any such Covered Software in Executable +form as to how they can obtain such Covered Software in Source Code form in a +reasonable manner on or through a medium customarily used for software +exchange. +3.2. Modifications. +The Modifications that You create or to which +You contribute are governed by the terms of this License. You represent that You +believe Your Modifications are Your original creation(s) and/or You have +sufficient rights to grant the rights conveyed by this License. +3.3. Required Notices. +You must include a notice in each of Your Modifications that +identifies You as the Contributor of the Modification. You may not remove or +alter any copyright, patent or trademark notices contained within the Covered +Software, or any notices of licensing or any descriptive text giving attribution +to any Contributor or the Initial Developer. +3.4. Application of Additional Terms. +You may not offer or impose any terms on any Covered Software in Source +Code form that alters or restricts the applicable version of this License or the +recipients' rights hereunder. You may choose to offer, and to charge a fee for, +warranty, support, indemnity or liability obligations to one or more recipients +of Covered Software. However, you may do so only on Your own behalf, and not on +behalf of the Initial Developer or any Contributor. You must make it absolutely +clear that any such warranty, support, indemnity or liability obligation is +offered by You alone, and You hereby agree to indemnify the Initial Developer +and every Contributor for any liability incurred by the Initial Developer or +such Contributor as a result of warranty, support, indemnity or liability terms +You offer. +3.5. Distribution of Executable Versions. +You may distribute the +Executable form of the Covered Software under the terms of this License or under +the terms of a license of Your choice, which may contain terms different from +this License, provided that You are in compliance with the terms of this License +and that the license for the Executable form does not attempt to limit or alter +the recipient's rights in the Source Code form from the rights set forth in +this License. If You distribute the Covered Software in Executable form under a +different license, You must make it absolutely clear that any terms which differ +from this License are offered by You alone, not by the Initial Developer or +Contributor. You hereby agree to indemnify the Initial Developer and every +Contributor for any liability incurred by the Initial Developer or such +Contributor as a result of any such terms You offer. +3.6. Larger Works. +You +may create a Larger Work by combining Covered Software with other code not +governed by the terms of this License and distribute the Larger Work as a single +product. In such a case, You must make sure the requirements of this License are +fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. +Oracle is the initial license steward and may publish revised and/or +new versions of this License from time to time. Each version will be given a +distinguishing version number. Except as provided in Section 4.3, no one other +than the license steward has the right to modify this License. +4.2. Effect of New Versions. +You may always continue to use, distribute or otherwise make the +Covered Software available under the terms of the version of the License under +which You originally received the Covered Software. If the Initial Developer +includes a notice in the Original Software prohibiting it from being distributed +or otherwise made available under any subsequent version of the License, You +must distribute and make the Covered Software available under the terms of the +version of the License under which You originally received the Covered Software. +Otherwise, You may also choose to use, distribute or otherwise make the Covered +Software available under the terms of any subsequent version of the License +published by the license steward. +4.3. Modified Versions. +When You are an +Initial Developer and You want to create a new license for Your Original +Software, You may create and use a modified version of this License if You: (a) +rename the license and remove any references to the name of the license steward +(except to note that the license differs from this License); and (b) otherwise +make it clear that the license contains terms which differ from this +License. + +COVERED SOFTWARE IS PROVIDED UNDER THIS +LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE +IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR +NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED +SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY +RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE +COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF +WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED +SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will +terminate automatically if You fail to comply with terms herein and fail to cure +such breach within 30 days of becoming aware of the breach. Provisions which, by +their nature, must remain in effect beyond the termination of this License shall +survive. +6.2. If You assert a patent infringement claim (excluding declaratory +judgment actions) against Initial Developer or a Contributor (the Initial +Developer or Contributor against whom You assert such claim is referred to as +“Participant”) alleging that the Participant Software (meaning the +Contributor Version where the Participant is a Contributor or the Original +Software where the Participant is the Initial Developer) directly or indirectly +infringes any patent, then any and all rights granted directly or indirectly to +You by such Participant, the Initial Developer (if the Initial Developer is not +the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this +License shall, upon 60 days notice from Participant terminate prospectively and +automatically at the expiration of such 60 day notice period, unless if within +such 60 day period You withdraw Your claim with respect to the Participant +Software against such Participant either unilaterally or pursuant to a written +agreement with Participant. +6.3. If You assert a patent infringement claim +against Participant alleging that the Participant Software directly or +indirectly infringes any patent where such claim is resolved (such as by license +or settlement) prior to the initiation of patent infringement litigation, then +the reasonable value of the licenses granted by such Participant under Sections +2.1 or 2.2 shall be taken into account in determining the amount or value of any +payment or license. +6.4. In the event of termination under Sections 6.1 or 6.2 +above, all end user licenses that have been validly granted by You or any +distributor hereunder prior to termination (excluding licenses granted to You by +any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT +(INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL +DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY +SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, +SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, +WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER +FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN +IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS +LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL +INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW +PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR +LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND +LIMITATION MAY NOT APPLY TO YOU. + +The Covered +Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 +(Oct. 1995), consisting of “commercial computer software” (as that term is +defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software +documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). +Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 +(June 1995), all U.S. Government End Users acquire Covered Software with only +those rights set forth herein. This U.S. Government Rights clause is in lieu of, +and supersedes, any other FAR, DFAR, or other clause or provision that addresses +Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning +subject matter hereof. If any provision of this License is held to be +unenforceable, such provision shall be reformed only to the extent necessary to +make it enforceable. This License shall be governed by the law of the +jurisdiction specified in a notice contained within the Original Software +(except to the extent applicable law, if any, provides otherwise), excluding +such jurisdiction's conflict-of-law provisions. Any litigation relating to this +License shall be subject to the jurisdiction of the courts located in the +jurisdiction and venue specified in a notice contained within the Original +Software, with the losing party responsible for costs, including, without +limitation, court costs and reasonable attorneys' fees and expenses. The +application of the United Nations Convention on Contracts for the International +Sale of Goods is expressly excluded. Any law or regulation which provides that +the language of a contract shall be construed against the drafter shall not +apply to this License. You agree that You alone are responsible for compliance +with the United States export administration regulations (and the export control +laws and regulation of any other countries) when You use, distribute or +otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is +responsible for claims and damages arising, directly or indirectly, out of its +utilization of rights under this License and You agree to work with Initial +Developer and Contributors to distribute such responsibility on an equitable +basis. Nothing herein is intended or shall be deemed to constitute any admission +of liability. + +The binary distribution of this product bundles these dependencies under the +following license: +Protocol Buffer Java API 2.5.0 +-------------------------------------------------------------------------------- +This license applies to all parts of Protocol Buffers except the following: + + - Atomicops support for generic gcc, located in + src/google/protobuf/stubs/atomicops_internals_generic_gcc.h. + This file is copyrighted by Red Hat Inc. + + - Atomicops support for AIX/POWER, located in + src/google/protobuf/stubs/atomicops_internals_power.h. + This file is copyrighted by Bloomberg Finance LP. + +Copyright 2014, Google Inc. All rights reserved. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +For: +XML Commons External Components XML APIs 1.3.04 +-------------------------------------------------------------------------------- +By obtaining, using and/or copying this work, you (the licensee) agree that you +have read, understood, and will comply with the following terms and conditions. + +Permission to copy, modify, and distribute this software and its documentation, +with or without modification, for any purpose and without fee or royalty is +hereby granted, provided that you include the following on ALL copies of the +software and documentation or portions thereof, including modifications: +- The full text of this NOTICE in a location viewable to users of the +redistributed or derivative work. +- Any pre-existing intellectual property disclaimers, notices, or terms and +conditions. If none exist, the W3C Software Short Notice should be included +(hypertext is preferred, text is permitted) within the body of any redistributed +or derivative code. +- Notice of any changes or modifications to the files, including the date changes +were made. (We recommend you provide URIs to the location from which the code is +derived.) + +The binary distribution of this product bundles these dependencies under the +following license: +JUnit 4.11 +Eclipse JDT Core 3.1.1 +-------------------------------------------------------------------------------- +(EPL v1.0) +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC +LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM +CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation +distributed under this Agreement, and +b) in the case of each subsequent Contributor: +i) changes to the Program, and +ii) additions to the Program; +where such changes and/or additions to the Program originate from and are +distributed by that particular Contributor. A Contribution 'originates' from a +Contributor if it was added to the Program by such Contributor itself or anyone +acting on such Contributor's behalf. Contributions do not include additions to +the Program which: (i) are separate modules of software distributed in +conjunction with the Program under their own license agreement, and (ii) are not +derivative works of the Program. +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are +necessarily infringed by the use or sale of its Contribution alone or when +combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, +including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free copyright license to +reproduce, prepare derivative works of, publicly display, publicly perform, +distribute and sublicense the Contribution of such Contributor, if any, and such +derivative works, in source code and object code form. +b) Subject to the terms of this Agreement, each Contributor hereby grants +Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed +Patents to make, use, sell, offer to sell, import and otherwise transfer the +Contribution of such Contributor, if any, in source code and object code form. +This patent license shall apply to the combination of the Contribution and the +Program if, at the time the Contribution is added by the Contributor, such +addition of the Contribution causes such combination to be covered by the +Licensed Patents. The patent license shall not apply to any other combinations +which include the Contribution. No hardware per se is licensed hereunder. +c) Recipient understands that although each Contributor grants the licenses to +its Contributions set forth herein, no assurances are provided by any +Contributor that the Program does not infringe the patent or other intellectual +property rights of any other entity. Each Contributor disclaims any liability to +Recipient for claims brought by any other entity based on infringement of +intellectual property rights or otherwise. As a condition to exercising the +rights and licenses granted hereunder, each Recipient hereby assumes sole +responsibility to secure any other intellectual property rights needed, if any. +For example, if a third party patent license is required to allow Recipient to +distribute the Program, it is Recipient's responsibility to acquire that license +before distributing the Program. +d) Each Contributor represents that to its knowledge it has sufficient copyright +rights in its Contribution, if any, to grant the copyright license set forth in +this Agreement. +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its +own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and +b) its license agreement: +i) effectively disclaims on behalf of all Contributors all warranties and +conditions, express and implied, including warranties or conditions of title and +non-infringement, and implied warranties or conditions of merchantability and +fitness for a particular purpose; +ii) effectively excludes on behalf of all Contributors all liability for +damages, including direct, indirect, special, incidental and consequential +damages, such as lost profits; +iii) states that any provisions which differ from this Agreement are offered by +that Contributor alone and not by any other party; and +iv) states that source code for the Program is available from such Contributor, +and informs licensees how to obtain it in a reasonable manner on or through a +medium customarily used for software exchange. +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and +b) a copy of this Agreement must be included with each copy of the Program. +Contributors may not remove or alter any copyright notices contained within the +Program. + +Each Contributor must identify itself as the originator of its Contribution, if +any, in a manner that reasonably allows subsequent Recipients to identify the +originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with +respect to end users, business partners and the like. While this license is +intended to facilitate the commercial use of the Program, the Contributor who +includes the Program in a commercial product offering should do so in a manner +which does not create potential liability for other Contributors. Therefore, if +a Contributor includes the Program in a commercial product offering, such +Contributor ("Commercial Contributor") hereby agrees to defend and indemnify +every other Contributor ("Indemnified Contributor") against any losses, damages +and costs (collectively "Losses") arising from claims, lawsuits and other legal +actions brought by a third party against the Indemnified Contributor to the +extent caused by the acts or omissions of such Commercial Contributor in +connection with its distribution of the Program in a commercial product +offering. The obligations in this section do not apply to any claims or Losses +relating to any actual or alleged intellectual property infringement. In order +to qualify, an Indemnified Contributor must: a) promptly notify the Commercial +Contributor in writing of such claim, and b) allow the Commercial Contributor to +control, and cooperate with the Commercial Contributor in, the defense and any +related settlement negotiations. The Indemnified Contributor may participate in +any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product +offering, Product X. That Contributor is then a Commercial Contributor. If that +Commercial Contributor then makes performance claims, or offers warranties +related to Product X, those performance claims and warranties are such +Commercial Contributor's responsibility alone. Under this section, the +Commercial Contributor would have to defend claims against the other +Contributors related to those performance claims and warranties, and if a court +requires any other Contributor to pay any damages as a result, the Commercial +Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR +IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, +NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each +Recipient is solely responsible for determining the appropriateness of using and +distributing the Program and assumes all risks associated with its exercise of +rights under this Agreement , including but not limited to the risks and costs +of program errors, compliance with applicable laws, damage to or loss of data, +programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY +CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST +PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS +GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this Agreement, and without further action by the parties hereto, such +provision shall be reformed to the minimum extent necessary to make such +provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a +cross-claim or counterclaim in a lawsuit) alleging that the Program itself +(excluding combinations of the Program with other software or hardware) +infringes such Recipient's patent(s), then such Recipient's rights granted under +Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to +comply with any of the material terms or conditions of this Agreement and does +not cure such failure in a reasonable period of time after becoming aware of +such noncompliance. If all Recipient's rights under this Agreement terminate, +Recipient agrees to cease use and distribution of the Program as soon as +reasonably practicable. However, Recipient's obligations under this Agreement +and any licenses granted by Recipient relating to the Program shall continue and +survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in +order to avoid inconsistency the Agreement is copyrighted and may only be +modified in the following manner. The Agreement Steward reserves the right to +publish new versions (including revisions) of this Agreement from time to time. +No one other than the Agreement Steward has the right to modify this Agreement. +The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation +may assign the responsibility to serve as the Agreement Steward to a suitable +separate entity. Each new version of the Agreement will be given a +distinguishing version number. The Program (including Contributions) may always +be distributed subject to the version of the Agreement under which it was +received. In addition, after a new version of the Agreement is published, +Contributor may elect to distribute the Program (including its Contributions) +under the new version. Except as expressly stated in Sections 2(a) and 2(b) +above, Recipient receives no rights or licenses to the intellectual property of +any Contributor under this Agreement, whether expressly, by implication, +estoppel or otherwise. All rights in the Program not expressly granted under +this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the +intellectual property laws of the United States of America. No party to this +Agreement will bring a legal action under this Agreement more than one year +after the cause of action arose. Each party waives its rights to a jury trial in +any resulting litigation. + +The binary distribution of this product bundles these dependencies under the +following license: +JSch 0.1.51 +ParaNamer Core 2.3 +JLine 0.9.94 +leveldbjni-all 1.8 +Hamcrest Core 1.3 +ASM Core 5.0.4 +ASM Commons 5.0.2 +ASM Tree 5.0.2 +-------------------------------------------------------------------------------- +(3-clause BSD) +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles these dependencies under the +following license: +FindBugs-jsr305 3.0.0 +dnsjava 2.1.7, Copyright (c) 1998-2011, Brian Wellington. All rights reserved. +-------------------------------------------------------------------------------- +(2-clause BSD) +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +The views and conclusions contained in the software and documentation are those +of the authors and should not be interpreted as representing official policies, +either expressed or implied, of the FreeBSD Project. + +The binary distribution of this product bundles these dependencies under the +following license: +"Java Concurrency in Practice" book annotations 1.0 +-------------------------------------------------------------------------------- +(CCAL v2.5) +THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE COMMONS +PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY COPYRIGHT AND/OR +OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS AUTHORIZED UNDER THIS +LICENSE OR COPYRIGHT LAW IS PROHIBITED. + +BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE TO BE +BOUND BY THE TERMS OF THIS LICENSE. THE LICENSOR GRANTS YOU THE RIGHTS CONTAINED +HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND CONDITIONS. + +1. Definitions + +"Collective Work" means a work, such as a periodical issue, anthology or +encyclopedia, in which the Work in its entirety in unmodified form, along with a +number of other contributions, constituting separate and independent works in +themselves, are assembled into a collective whole. A work that constitutes a +Collective Work will not be considered a Derivative Work (as defined below) for +the purposes of this License. +"Derivative Work" means a work based upon the Work or upon the Work and other +pre-existing works, such as a translation, musical arrangement, dramatization, +fictionalization, motion picture version, sound recording, art reproduction, +abridgment, condensation, or any other form in which the Work may be recast, +transformed, or adapted, except that a work that constitutes a Collective Work +will not be considered a Derivative Work for the purpose of this License. For +the avoidance of doubt, where the Work is a musical composition or sound +recording, the synchronization of the Work in timed-relation with a moving image +("synching") will be considered a Derivative Work for the purpose of this +License. +"Licensor" means the individual or entity that offers the Work under the terms +of this License. +"Original Author" means the individual or entity who created the Work. +"Work" means the copyrightable work of authorship offered under the terms of +this License. +"You" means an individual or entity exercising rights under this License who has +not previously violated the terms of this License with respect to the Work, or +who has received express permission from the Licensor to exercise rights under +this License despite a previous violation. +2. Fair Use Rights. Nothing in this license is intended to reduce, limit, or +restrict any rights arising from fair use, first sale or other limitations on +the exclusive rights of the copyright owner under copyright law or other +applicable laws. + +3. License Grant. Subject to the terms and conditions of this License, Licensor +hereby grants You a worldwide, royalty-free, non-exclusive, perpetual (for the +duration of the applicable copyright) license to exercise the rights in the Work +as stated below: + +to reproduce the Work, to incorporate the Work into one or more Collective +Works, and to reproduce the Work as incorporated in the Collective Works; +to create and reproduce Derivative Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission the Work including as +incorporated in Collective Works; +to distribute copies or phonorecords of, display publicly, perform publicly, and +perform publicly by means of a digital audio transmission Derivative Works. +For the avoidance of doubt, where the work is a musical composition: + +Performance Royalties Under Blanket Licenses. Licensor waives the exclusive +right to collect, whether individually or via a performance rights society (e.g. +ASCAP, BMI, SESAC), royalties for the public performance or public digital +performance (e.g. webcast) of the Work. +Mechanical Rights and Statutory Royalties. Licensor waives the exclusive right +to collect, whether individually or via a music rights agency or designated +agent (e.g. Harry Fox Agency), royalties for any phonorecord You create from the +Work ("cover version") and distribute, subject to the compulsory license created +by 17 USC Section 115 of the US Copyright Act (or the equivalent in other +jurisdictions). +Webcasting Rights and Statutory Royalties. For the avoidance of doubt, where the +Work is a sound recording, Licensor waives the exclusive right to collect, +whether individually or via a performance-rights society (e.g. SoundExchange), +royalties for the public digital performance (e.g. webcast) of the Work, subject +to the compulsory license created by 17 USC Section 114 of the US Copyright Act +(or the equivalent in other jurisdictions). +The above rights may be exercised in all media and formats whether now known or +hereafter devised. The above rights include the right to make such modifications +as are technically necessary to exercise the rights in other media and formats. +All rights not expressly granted by Licensor are hereby reserved. + +4. Restrictions.The license granted in Section 3 above is expressly made subject +to and limited by the following restrictions: + +You may distribute, publicly display, publicly perform, or publicly digitally +perform the Work only under the terms of this License, and You must include a +copy of, or the Uniform Resource Identifier for, this License with every copy or +phonorecord of the Work You distribute, publicly display, publicly perform, or +publicly digitally perform. You may not offer or impose any terms on the Work +that alter or restrict the terms of this License or the recipients' exercise of +the rights granted hereunder. You may not sublicense the Work. You must keep +intact all notices that refer to this License and to the disclaimer of +warranties. You may not distribute, publicly display, publicly perform, or +publicly digitally perform the Work with any technological measures that control +access or use of the Work in a manner inconsistent with the terms of this +License Agreement. The above applies to the Work as incorporated in a Collective +Work, but this does not require the Collective Work apart from the Work itself +to be made subject to the terms of this License. If You create a Collective +Work, upon notice from any Licensor You must, to the extent practicable, remove +from the Collective Work any credit as required by clause 4(b), as requested. If +You create a Derivative Work, upon notice from any Licensor You must, to the +extent practicable, remove from the Derivative Work any credit as required by +clause 4(b), as requested. +If you distribute, publicly display, publicly perform, or publicly digitally +perform the Work or any Derivative Works or Collective Works, You must keep +intact all copyright notices for the Work and provide, reasonable to the medium +or means You are utilizing: (i) the name of the Original Author (or pseudonym, +if applicable) if supplied, and/or (ii) if the Original Author and/or Licensor +designate another party or parties (e.g. a sponsor institute, publishing entity, +journal) for attribution in Licensor's copyright notice, terms of service or by +other reasonable means, the name of such party or parties; the title of the Work +if supplied; to the extent reasonably practicable, the Uniform Resource +Identifier, if any, that Licensor specifies to be associated with the Work, +unless such URI does not refer to the copyright notice or licensing information +for the Work; and in the case of a Derivative Work, a credit identifying the use +of the Work in the Derivative Work (e.g., "French translation of the Work by +Original Author," or "Screenplay based on original Work by Original Author"). +Such credit may be implemented in any reasonable manner; provided, however, that +in the case of a Derivative Work or Collective Work, at a minimum such credit +will appear where any other comparable authorship credit appears and in a manner +at least as prominent as such other comparable authorship credit. +5. Representations, Warranties and Disclaimer + +UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR OFFERS +THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY KIND CONCERNING +THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE, INCLUDING, WITHOUT +LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY, FITNESS FOR A PARTICULAR +PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF LATENT OR OTHER DEFECTS, ACCURACY, +OR THE PRESENCE OF ABSENCE OF ERRORS, WHETHER OR NOT DISCOVERABLE. SOME +JURISDICTIONS DO NOT ALLOW THE EXCLUSION OF IMPLIED WARRANTIES, SO SUCH +EXCLUSION MAY NOT APPLY TO YOU. + +6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE LAW, IN +NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR ANY SPECIAL, +INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES ARISING OUT OF THIS +LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + +7. Termination + +This License and the rights granted hereunder will terminate automatically upon +any breach by You of the terms of this License. Individuals or entities who have +received Derivative Works or Collective Works from You under this License, +however, will not have their licenses terminated provided such individuals or +entities remain in full compliance with those licenses. Sections 1, 2, 5, 6, 7, +and 8 will survive any termination of this License. +Subject to the above terms and conditions, the license granted here is perpetual +(for the duration of the applicable copyright in the Work). Notwithstanding the +above, Licensor reserves the right to release the Work under different license +terms or to stop distributing the Work at any time; provided, however that any +such election will not serve to withdraw this License (or any other license that +has been, or is required to be, granted under the terms of this License), and +this License will continue in full force and effect unless terminated as stated +above. +8. Miscellaneous + +Each time You distribute or publicly digitally perform the Work or a Collective +Work, the Licensor offers to the recipient a license to the Work on the same +terms and conditions as the license granted to You under this License. +Each time You distribute or publicly digitally perform a Derivative Work, +Licensor offers to the recipient a license to the original Work on the same +terms and conditions as the license granted to You under this License. +If any provision of this License is invalid or unenforceable under applicable +law, it shall not affect the validity or enforceability of the remainder of the +terms of this License, and without further action by the parties to this +agreement, such provision shall be reformed to the minimum extent necessary to +make such provision valid and enforceable. +No term or provision of this License shall be deemed waived and no breach +consented to unless such waiver or consent shall be in writing and signed by the +party to be charged with such waiver or consent. +This License constitutes the entire agreement between the parties with respect +to the Work licensed here. There are no understandings, agreements or +representations with respect to the Work not specified here. Licensor shall not +be bound by any additional provisions that may appear in any communication from +You. This License may not be modified without the mutual written agreement of +the Licensor and You. + +The binary distribution of this product bundles these dependencies under the +following license: +jamon-runtime 2.4.1 +-------------------------------------------------------------------------------- +(MPL 2.0) + Mozilla Public License + Version 2.0 + +1.1. “Contributor” +means each individual or legal entity that creates, contributes to the creation +of, or owns Covered Software. + +1.2. “Contributor Version” +means the combination of the Contributions of others (if any) used by a +Contributor and that particular Contributor’s Contribution. + +1.3. “Contribution” +means Covered Software of a particular Contributor. + +1.4. “Covered Software” +means Source Code Form to which the initial Contributor has attached the notice +in Exhibit A, the Executable Form of such Source Code Form, and Modifications of +such Source Code Form, in each case including portions thereof. + +1.5. “Incompatible With Secondary Licenses” +means + +that the initial Contributor has attached the notice described in Exhibit B to +the Covered Software; or + +that the Covered Software was made available under the terms of version 1.1 or +earlier of the License, but not also under the terms of a Secondary License. + +1.6. “Executable Form” +means any form of the work other than Source Code Form. + +1.7. “Larger Work” +means a work that combines Covered Software with other material, in a separate +file or files, that is not Covered Software. + +1.8. “License” +means this document. + +1.9. “Licensable” +means having the right to grant, to the maximum extent possible, whether at the +time of the initial grant or subsequently, any and all of the rights conveyed by +this License. + +1.10. “Modifications” +means any of the following: + +any file in Source Code Form that results from an addition to, deletion from, or +modification of the contents of Covered Software; or + +any new file in Source Code Form that contains any Covered Software. + +1.11. “Patent Claims” of a Contributor +means any patent claim(s), including without limitation, method, process, and +apparatus claims, in any patent Licensable by such Contributor that would be +infringed, but for the grant of the License, by the making, using, selling, +offering for sale, having made, import, or transfer of either its Contributions +or its Contributor Version. + +1.12. “Secondary License” +means either the GNU General Public License, Version 2.0, the GNU Lesser General +Public License, Version 2.1, the GNU Affero General Public License, Version 3.0, +or any later versions of those licenses. + +1.13. “Source Code Form” +means the form of the work preferred for making modifications. + +1.14. “You” (or “Your”) +means an individual or a legal entity exercising rights under this License. For +legal entities, “You” includes any entity that controls, is controlled by, +or is under common control with You. For purposes of this definition, +“control” means (a) the power, direct or indirect, to cause the direction or +management of such entity, whether by contract or otherwise, or (b) ownership of +more than fifty percent (50%) of the outstanding shares or beneficial ownership +of such entity. + +2. License Grants and Conditions + +2.1. Grants + +Each Contributor hereby grants You a world-wide, royalty-free, non-exclusive +license: + +under intellectual property rights (other than patent or trademark) Licensable +by such Contributor to use, reproduce, make available, modify, display, perform, +distribute, and otherwise exploit its Contributions, either on an unmodified +basis, with Modifications, or as part of a Larger Work; and + +under Patent Claims of such Contributor to make, use, sell, offer for sale, have +made, import, and otherwise transfer either its Contributions or its Contributor +Version. + +2.2. Effective Date + +The licenses granted in Section 2.1 with respect to any Contribution become +effective for each Contribution on the date the Contributor first distributes +such Contribution. + +2.3. Limitations on Grant Scope + +The licenses granted in this Section 2 are the only rights granted under this +License. No additional rights or licenses will be implied from the distribution +or licensing of Covered Software under this License. Notwithstanding Section +2.1(b) above, no patent license is granted by a Contributor: + +for any code that a Contributor has removed from Covered Software; or + +for infringements caused by: (i) Your and any other third party’s +modifications of Covered Software, or (ii) the combination of its Contributions +with other software (except as part of its Contributor Version); or + +under Patent Claims infringed by Covered Software in the absence of its +Contributions. + +This License does not grant any rights in the trademarks, service marks, or +logos of any Contributor (except as may be necessary to comply with the notice +requirements in Section 3.4). + +2.4. Subsequent Licenses + +No Contributor makes additional grants as a result of Your choice to distribute +the Covered Software under a subsequent version of this License (see Section +10.2) or under the terms of a Secondary License (if permitted under the terms of +Section 3.3). + +2.5. Representation + +Each Contributor represents that the Contributor believes its Contributions are +its original creation(s) or it has sufficient rights to grant the rights to its +Contributions conveyed by this License. + +2.6. Fair Use + +This License is not intended to limit any rights You have under applicable +copyright doctrines of fair use, fair dealing, or other equivalents. + +2.7. Conditions + +Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted in +Section 2.1. + +3. Responsibilities + +3.1. Distribution of Source Form + +All distribution of Covered Software in Source Code Form, including any +Modifications that You create or to which You contribute, must be under the +terms of this License. You must inform recipients that the Source Code Form of +the Covered Software is governed by the terms of this License, and how they can +obtain a copy of this License. You may not attempt to alter or restrict the +recipients’ rights in the Source Code Form. + +3.2. Distribution of Executable Form + +If You distribute Covered Software in Executable Form then: + +such Covered Software must also be made available in Source Code Form, as +described in Section 3.1, and You must inform recipients of the Executable Form +how they can obtain a copy of such Source Code Form by reasonable means in a +timely manner, at a charge no more than the cost of distribution to the +recipient; and + +You may distribute such Executable Form under the terms of this License, or +sublicense it under different terms, provided that the license for the +Executable Form does not attempt to limit or alter the recipients’ rights in +the Source Code Form under this License. + +3.3. Distribution of a Larger Work + +You may create and distribute a Larger Work under terms of Your choice, provided +that You also comply with the requirements of this License for the Covered +Software. If the Larger Work is a combination of Covered Software with a work +governed by one or more Secondary Licenses, and the Covered Software is not +Incompatible With Secondary Licenses, this License permits You to additionally +distribute such Covered Software under the terms of such Secondary License(s), +so that the recipient of the Larger Work may, at their option, further +distribute the Covered Software under the terms of either this License or such +Secondary License(s). + +3.4. Notices + +You may not remove or alter the substance of any license notices (including +copyright notices, patent notices, disclaimers of warranty, or limitations of +liability) contained within the Source Code Form of the Covered Software, except +that You may alter any license notices to the extent required to remedy known +factual inaccuracies. + +3.5. Application of Additional Terms + +You may choose to offer, and to charge a fee for, warranty, support, indemnity +or liability obligations to one or more recipients of Covered Software. However, +You may do so only on Your own behalf, and not on behalf of any Contributor. You +must make it absolutely clear that any such warranty, support, indemnity, or +liability obligation is offered by You alone, and You hereby agree to indemnify +every Contributor for any liability incurred by such Contributor as a result of +warranty, support, indemnity or liability terms You offer. You may include +additional disclaimers of warranty and limitations of liability specific to any +jurisdiction. + +4. Inability to Comply Due to Statute or Regulation + +If it is impossible for You to comply with any of the terms of this License with +respect to some or all of the Covered Software due to statute, judicial order, +or regulation then You must: (a) comply with the terms of this License to the +maximum extent possible; and (b) describe the limitations and the code they +affect. Such description must be placed in a text file included with all +distributions of the Covered Software under this License. Except to the extent +prohibited by statute or regulation, such description must be sufficiently +detailed for a recipient of ordinary skill to be able to understand it. + +5. Termination + +5.1. The rights granted under this License will terminate automatically if You +fail to comply with any of its terms. However, if You become compliant, then the +rights granted under this License from a particular Contributor are reinstated +(a) provisionally, unless and until such Contributor explicitly and finally +terminates Your grants, and (b) on an ongoing basis, if such Contributor fails +to notify You of the non-compliance by some reasonable means prior to 60 days +after You have come back into compliance. Moreover, Your grants from a +particular Contributor are reinstated on an ongoing basis if such Contributor +notifies You of the non-compliance by some reasonable means, this is the first +time You have received notice of non-compliance with this License from such +Contributor, and You become compliant prior to 30 days after Your receipt of the +notice. + +5.2. If You initiate litigation against any entity by asserting a patent +infringement claim (excluding declaratory judgment actions, counter-claims, and +cross-claims) alleging that a Contributor Version directly or indirectly +infringes any patent, then the rights granted to You by any and all Contributors +for the Covered Software under Section 2.1 of this License shall terminate. + +5.3. In the event of termination under Sections 5.1 or 5.2 above, all end user +license agreements (excluding distributors and resellers) which have been +validly granted by You or Your distributors under this License prior to +termination shall survive termination. + +6. Disclaimer of Warranty + +Covered Software is provided under this License on an “as is” basis, without +warranty of any kind, either expressed, implied, or statutory, including, +without limitation, warranties that the Covered Software is free of defects, +merchantable, fit for a particular purpose or non-infringing. The entire risk as +to the quality and performance of the Covered Software is with You. Should any +Covered Software prove defective in any respect, You (not any Contributor) +assume the cost of any necessary servicing, repair, or correction. This +disclaimer of warranty constitutes an essential part of this License. No use of +any Covered Software is authorized under this License except under this +disclaimer. + +7. Limitation of Liability + +Under no circumstances and under no legal theory, whether tort (including +negligence), contract, or otherwise, shall any Contributor, or anyone who +distributes Covered Software as permitted above, be liable to You for any +direct, indirect, special, incidental, or consequential damages of any character +including, without limitation, damages for lost profits, loss of goodwill, work +stoppage, computer failure or malfunction, or any and all other commercial +damages or losses, even if such party shall have been informed of the +possibility of such damages. This limitation of liability shall not apply to +liability for death or personal injury resulting from such party’s negligence +to the extent applicable law prohibits such limitation. Some jurisdictions do +not allow the exclusion or limitation of incidental or consequential damages, so +this exclusion and limitation may not apply to You. + +8. Litigation + +Any litigation relating to this License may be brought only in the courts of a +jurisdiction where the defendant maintains its principal place of business and +such litigation shall be governed by laws of that jurisdiction, without +reference to its conflict-of-law provisions. Nothing in this Section shall +prevent a party’s ability to bring cross-claims or counter-claims. + +9. Miscellaneous + +This License represents the complete agreement concerning the subject matter +hereof. If any provision of this License is held to be unenforceable, such +provision shall be reformed only to the extent necessary to make it enforceable. +Any law or regulation which provides that the language of a contract shall be +construed against the drafter shall not be used to construe this License against +a Contributor. + +10. Versions of the License + +10.1. New Versions + +Mozilla Foundation is the license steward. Except as provided in Section 10.3, +no one other than the license steward has the right to modify or publish new +versions of this License. Each version will be given a distinguishing version +number. + +10.2. Effect of New Versions + +You may distribute the Covered Software under the terms of the version of the +License under which You originally received the Covered Software, or under the +terms of any subsequent version published by the license steward. + +10.3. Modified Versions + +If you create software not governed by this License, and you want to create a +new license for such software, you may create and use a modified version of this +License if you rename the license and remove any references to the name of the +license steward (except to note that such modified license differs from this +License). + +10.4. Distributing Source Code Form that is Incompatible With Secondary Licenses + +If You choose to distribute Source Code Form that is Incompatible With Secondary +Licenses under the terms of this version of the License, the notice described in +Exhibit B of this License must be attached. + +Exhibit A - Source Code Form License Notice + +This Source Code Form is subject to the terms of the Mozilla Public License, v. +2.0. If a copy of the MPL was not distributed with this file, You can obtain one +at https://mozilla.org/MPL/2.0/. + +If it is not possible or desirable to put the notice in a particular file, then +You may include the notice in a location (such as a LICENSE file in a relevant +directory) where a recipient would be likely to look for such a notice. + +You may add additional accurate notices of copyright ownership. + +Exhibit B - “Incompatible With Secondary Licenses” Notice + +This Source Code Form is “Incompatible With Secondary Licenses”, as defined +by the Mozilla Public License, v. 2.0. + +The binary distribution of this product bundles these dependencies under the +following license: +JDOM 1.1 +-------------------------------------------------------------------------------- +/*-- + + Copyright (C) 2000-2004 Jason Hunter & Brett McLaughlin. + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions, and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions, and the disclaimer that follows + these conditions in the documentation and/or other materials + provided with the distribution. + + 3. The name "JDOM" must not be used to endorse or promote products + derived from this software without prior written permission. For + written permission, please contact . + + 4. Products derived from this software may not be called "JDOM", nor + may "JDOM" appear in their name, without prior written permission + from the JDOM Project Management . + + In addition, we request (but do not require) that you include in the + end-user documentation provided with the redistribution and/or in the + software itself an acknowledgement equivalent to the following: + "This product includes software developed by the + JDOM Project (http://www.jdom.org/)." + Alternatively, the acknowledgment may be graphical using the logos + available at http://www.jdom.org/images/logos. + + THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED + WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE JDOM AUTHORS OR THE PROJECT + CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF + USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND + ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, + OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT + OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + SUCH DAMAGE. + + This software consists of voluntary contributions made by many + individuals on behalf of the JDOM Project and was originally + created by Jason Hunter and + Brett McLaughlin . For more information + on the JDOM Project, please see . + + */ + +The binary distribution of this product bundles these dependencies under the +following license: +Hbase Server 1.2.4 +-------------------------------------------------------------------------------- +This project bundles a derivative image for our Orca Logo. This image is +available under the Creative Commons By Attribution 3.0 License. + + Creative Commons Legal Code + + Attribution 3.0 Unported + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS LICENSE DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE INFORMATION PROVIDED, AND DISCLAIMS LIABILITY FOR + DAMAGES RESULTING FROM ITS USE. + + License + + THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE + COMMONS PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY + COPYRIGHT AND/OR OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS + AUTHORIZED UNDER THIS LICENSE OR COPYRIGHT LAW IS PROHIBITED. + + BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE + TO BE BOUND BY THE TERMS OF THIS LICENSE. TO THE EXTENT THIS LICENSE MAY + BE CONSIDERED TO BE A CONTRACT, THE LICENSOR GRANTS YOU THE RIGHTS + CONTAINED HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND + CONDITIONS. + + 1. Definitions + + a. "Adaptation" means a work based upon the Work, or upon the Work and + other pre-existing works, such as a translation, adaptation, + derivative work, arrangement of music or other alterations of a + literary or artistic work, or phonogram or performance and includes + cinematographic adaptations or any other form in which the Work may be + recast, transformed, or adapted including in any form recognizably + derived from the original, except that a work that constitutes a + Collection will not be considered an Adaptation for the purpose of + this License. For the avoidance of doubt, where the Work is a musical + work, performance or phonogram, the synchronization of the Work in + timed-relation with a moving image ("synching") will be considered an + Adaptation for the purpose of this License. + b. "Collection" means a collection of literary or artistic works, such as + encyclopedias and anthologies, or performances, phonograms or + broadcasts, or other works or subject matter other than works listed + in Section 1(f) below, which, by reason of the selection and + arrangement of their contents, constitute intellectual creations, in + which the Work is included in its entirety in unmodified form along + with one or more other contributions, each constituting separate and + independent works in themselves, which together are assembled into a + collective whole. A work that constitutes a Collection will not be + considered an Adaptation (as defined above) for the purposes of this + License. + c. "Distribute" means to make available to the public the original and + copies of the Work or Adaptation, as appropriate, through sale or + other transfer of ownership. + d. "Licensor" means the individual, individuals, entity or entities that + offer(s) the Work under the terms of this License. + e. "Original Author" means, in the case of a literary or artistic work, + the individual, individuals, entity or entities who created the Work + or if no individual or entity can be identified, the publisher; and in + addition (i) in the case of a performance the actors, singers, + musicians, dancers, and other persons who act, sing, deliver, declaim, + play in, interpret or otherwise perform literary or artistic works or + expressions of folklore; (ii) in the case of a phonogram the producer + being the person or legal entity who first fixes the sounds of a + performance or other sounds; and, (iii) in the case of broadcasts, the + organization that transmits the broadcast. + f. "Work" means the literary and/or artistic work offered under the terms + of this License including without limitation any production in the + literary, scientific and artistic domain, whatever may be the mode or + form of its expression including digital form, such as a book, + pamphlet and other writing; a lecture, address, sermon or other work + of the same nature; a dramatic or dramatico-musical work; a + choreographic work or entertainment in dumb show; a musical + composition with or without words; a cinematographic work to which are + assimilated works expressed by a process analogous to cinematography; + a work of drawing, painting, architecture, sculpture, engraving or + lithography; a photographic work to which are assimilated works + expressed by a process analogous to photography; a work of applied + art; an illustration, map, plan, sketch or three-dimensional work + relative to geography, topography, architecture or science; a + performance; a broadcast; a phonogram; a compilation of data to the + extent it is protected as a copyrightable work; or a work performed by + a variety or circus performer to the extent it is not otherwise + considered a literary or artistic work. + g. "You" means an individual or entity exercising rights under this + License who has not previously violated the terms of this License with + respect to the Work, or who has received express permission from the + Licensor to exercise rights under this License despite a previous + violation. + h. "Publicly Perform" means to perform public recitations of the Work and + to communicate to the public those public recitations, by any means or + process, including by wire or wireless means or public digital + performances; to make available to the public Works in such a way that + members of the public may access these Works from a place and at a + place individually chosen by them; to perform the Work to the public + by any means or process and the communication to the public of the + performances of the Work, including by public digital performance; to + broadcast and rebroadcast the Work by any means including signs, + sounds or images. + i. "Reproduce" means to make copies of the Work by any means including + without limitation by sound or visual recordings and the right of + fixation and reproducing fixations of the Work, including storage of a + protected performance or phonogram in digital form or other electronic + medium. + + 2. Fair Dealing Rights. Nothing in this License is intended to reduce, + limit, or restrict any uses free from copyright or rights arising from + limitations or exceptions that are provided for in connection with the + copyright protection under copyright law or other applicable laws. + + 3. License Grant. Subject to the terms and conditions of this License, + Licensor hereby grants You a worldwide, royalty-free, non-exclusive, + perpetual (for the duration of the applicable copyright) license to + exercise the rights in the Work as stated below: + + a. to Reproduce the Work, to incorporate the Work into one or more + Collections, and to Reproduce the Work as incorporated in the + Collections; + b. to create and Reproduce Adaptations provided that any such Adaptation, + including any translation in any medium, takes reasonable steps to + clearly label, demarcate or otherwise identify that changes were made + to the original Work. For example, a translation could be marked "The + original work was translated from English to Spanish," or a + modification could indicate "The original work has been modified."; + c. to Distribute and Publicly Perform the Work including as incorporated + in Collections; and, + d. to Distribute and Publicly Perform Adaptations. + e. For the avoidance of doubt: + + i. Non-waivable Compulsory License Schemes. In those jurisdictions in + which the right to collect royalties through any statutory or + compulsory licensing scheme cannot be waived, the Licensor + reserves the exclusive right to collect such royalties for any + exercise by You of the rights granted under this License; + ii. Waivable Compulsory License Schemes. In those jurisdictions in + which the right to collect royalties through any statutory or + compulsory licensing scheme can be waived, the Licensor waives the + exclusive right to collect such royalties for any exercise by You + of the rights granted under this License; and, + iii. Voluntary License Schemes. The Licensor waives the right to + collect royalties, whether individually or, in the event that the + Licensor is a member of a collecting society that administers + voluntary licensing schemes, via that society, from any exercise + by You of the rights granted under this License. + + The above rights may be exercised in all media and formats whether now + known or hereafter devised. The above rights include the right to make + such modifications as are technically necessary to exercise the rights in + other media and formats. Subject to Section 8(f), all rights not expressly + granted by Licensor are hereby reserved. + + 4. Restrictions. The license granted in Section 3 above is expressly made + subject to and limited by the following restrictions: + + a. You may Distribute or Publicly Perform the Work only under the terms + of this License. You must include a copy of, or the Uniform Resource + Identifier (URI) for, this License with every copy of the Work You + Distribute or Publicly Perform. You may not offer or impose any terms + on the Work that restrict the terms of this License or the ability of + the recipient of the Work to exercise the rights granted to that + recipient under the terms of the License. You may not sublicense the + Work. You must keep intact all notices that refer to this License and + to the disclaimer of warranties with every copy of the Work You + Distribute or Publicly Perform. When You Distribute or Publicly + Perform the Work, You may not impose any effective technological + measures on the Work that restrict the ability of a recipient of the + Work from You to exercise the rights granted to that recipient under + the terms of the License. This Section 4(a) applies to the Work as + incorporated in a Collection, but this does not require the Collection + apart from the Work itself to be made subject to the terms of this + License. If You create a Collection, upon notice from any Licensor You + must, to the extent practicable, remove from the Collection any credit + as required by Section 4(b), as requested. If You create an + Adaptation, upon notice from any Licensor You must, to the extent + practicable, remove from the Adaptation any credit as required by + Section 4(b), as requested. + b. If You Distribute, or Publicly Perform the Work or any Adaptations or + Collections, You must, unless a request has been made pursuant to + Section 4(a), keep intact all copyright notices for the Work and + provide, reasonable to the medium or means You are utilizing: (i) the + name of the Original Author (or pseudonym, if applicable) if supplied, + and/or if the Original Author and/or Licensor designate another party + or parties (e.g., a sponsor institute, publishing entity, journal) for + attribution ("Attribution Parties") in Licensor's copyright notice, + terms of service or by other reasonable means, the name of such party + or parties; (ii) the title of the Work if supplied; (iii) to the + extent reasonably practicable, the URI, if any, that Licensor + specifies to be associated with the Work, unless such URI does not + refer to the copyright notice or licensing information for the Work; + and (iv) , consistent with Section 3(b), in the case of an Adaptation, + a credit identifying the use of the Work in the Adaptation (e.g., + "French translation of the Work by Original Author," or "Screenplay + based on original Work by Original Author"). The credit required by + this Section 4 (b) may be implemented in any reasonable manner; + provided, however, that in the case of a Adaptation or Collection, at + a minimum such credit will appear, if a credit for all contributing + authors of the Adaptation or Collection appears, then as part of these + credits and in a manner at least as prominent as the credits for the + other contributing authors. For the avoidance of doubt, You may only + use the credit required by this Section for the purpose of attribution + in the manner set out above and, by exercising Your rights under this + License, You may not implicitly or explicitly assert or imply any + connection with, sponsorship or endorsement by the Original Author, + Licensor and/or Attribution Parties, as appropriate, of You or Your + use of the Work, without the separate, express prior written + permission of the Original Author, Licensor and/or Attribution + Parties. + c. Except as otherwise agreed in writing by the Licensor or as may be + otherwise permitted by applicable law, if You Reproduce, Distribute or + Publicly Perform the Work either by itself or as part of any + Adaptations or Collections, You must not distort, mutilate, modify or + take other derogatory action in relation to the Work which would be + prejudicial to the Original Author's honor or reputation. Licensor + agrees that in those jurisdictions (e.g. Japan), in which any exercise + of the right granted in Section 3(b) of this License (the right to + make Adaptations) would be deemed to be a distortion, mutilation, + modification or other derogatory action prejudicial to the Original + Author's honor and reputation, the Licensor will waive or not assert, + as appropriate, this Section, to the fullest extent permitted by the + applicable national law, to enable You to reasonably exercise Your + right under Section 3(b) of this License (right to make Adaptations) + but not otherwise. + + 5. Representations, Warranties and Disclaimer + + UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR + OFFERS THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY + KIND CONCERNING THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE, + INCLUDING, WITHOUT LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY, + FITNESS FOR A PARTICULAR PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF + LATENT OR OTHER DEFECTS, ACCURACY, OR THE PRESENCE OF ABSENCE OF ERRORS, + WHETHER OR NOT DISCOVERABLE. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION + OF IMPLIED WARRANTIES, SO SUCH EXCLUSION MAY NOT APPLY TO YOU. + + 6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE + LAW, IN NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR + ANY SPECIAL, INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES + ARISING OUT OF THIS LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS + BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + + 7. Termination + + a. This License and the rights granted hereunder will terminate + automatically upon any breach by You of the terms of this License. + Individuals or entities who have received Adaptations or Collections + from You under this License, however, will not have their licenses + terminated provided such individuals or entities remain in full + compliance with those licenses. Sections 1, 2, 5, 6, 7, and 8 will + survive any termination of this License. + b. Subject to the above terms and conditions, the license granted here is + perpetual (for the duration of the applicable copyright in the Work). + Notwithstanding the above, Licensor reserves the right to release the + Work under different license terms or to stop distributing the Work at + any time; provided, however that any such election will not serve to + withdraw this License (or any other license that has been, or is + required to be, granted under the terms of this License), and this + License will continue in full force and effect unless terminated as + stated above. + + 8. Miscellaneous + + a. Each time You Distribute or Publicly Perform the Work or a Collection, + the Licensor offers to the recipient a license to the Work on the same + terms and conditions as the license granted to You under this License. + b. Each time You Distribute or Publicly Perform an Adaptation, Licensor + offers to the recipient a license to the original Work on the same + terms and conditions as the license granted to You under this License. + c. If any provision of this License is invalid or unenforceable under + applicable law, it shall not affect the validity or enforceability of + the remainder of the terms of this License, and without further action + by the parties to this agreement, such provision shall be reformed to + the minimum extent necessary to make such provision valid and + enforceable. + d. No term or provision of this License shall be deemed waived and no + breach consented to unless such waiver or consent shall be in writing + and signed by the party to be charged with such waiver or consent. + e. This License constitutes the entire agreement between the parties with + respect to the Work licensed here. There are no understandings, + agreements or representations with respect to the Work not specified + here. Licensor shall not be bound by any additional provisions that + may appear in any communication from You. This License may not be + modified without the mutual written agreement of the Licensor and You. + f. The rights granted under, and the subject matter referenced, in this + License were drafted utilizing the terminology of the Berne Convention + for the Protection of Literary and Artistic Works (as amended on + September 28, 1979), the Rome Convention of 1961, the WIPO Copyright + Treaty of 1996, the WIPO Performances and Phonograms Treaty of 1996 + and the Universal Copyright Convention (as revised on July 24, 1971). + These rights and subject matter take effect in the relevant + jurisdiction in which the License terms are sought to be enforced + according to the corresponding provisions of the implementation of + those treaty provisions in the applicable national law. If the + standard suite of rights granted under applicable copyright law + includes additional rights not granted under this License, such + additional rights are deemed to be included in the License; this + License is not intended to restrict the license of any rights under + applicable law. + + Creative Commons Notice + + Creative Commons is not a party to this License, and makes no warranty + whatsoever in connection with the Work. Creative Commons will not be + liable to You or any party on any legal theory for any damages + whatsoever, including without limitation any general, special, + incidental or consequential damages arising in connection to this + license. Notwithstanding the foregoing two (2) sentences, if Creative + Commons has expressly identified itself as the Licensor hereunder, it + shall have all rights and obligations of Licensor. + + Except for the limited purpose of indicating to the public that the + Work is licensed under the CCPL, Creative Commons does not authorize + the use by either party of the trademark "Creative Commons" or any + related trademark or logo of Creative Commons without the prior + written consent of Creative Commons. Any permitted use will be in + compliance with Creative Commons' then-current trademark usage + guidelines, as may be published on its website or otherwise made + available upon request from time to time. For the avoidance of doubt, + this trademark restriction does not form part of this License. + + Creative Commons may be contacted at https://creativecommons.org/. +-------------------------------------------------------------------------------- + +For: hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs +/server/datanode/checker/AbstractFuture.java and +hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs +/server/datanode/checker/TimeoutFuture.java + +Copyright (C) 2007 The Guava Authors + +Licensed 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. + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +The binary distribution of this product bundles binaries of +org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the +following notices: +* Copyright 2011 Dain Sundstrom +* Copyright 2011 FuseSource Corp. http://fusesource.com + +The binary distribution of this product bundles binaries of +AWS SDK for Java - Bundle 1.11.134, +AWS Java SDK for AWS KMS 1.11.134, +AWS Java SDK for Amazon S3 1.11.134, +AWS Java SDK for AWS STS 1.11.134, +JMES Path Query library 1.0, +which has the following notices: + * This software includes third party software subject to the following + copyrights: - XML parsing and utility functions from JetS3t - Copyright + 2006-2009 James Murty. - JSON parsing and utility functions from JSON.org - + Copyright 2002 JSON.org. - PKCS#1 PEM encoded private key parsing and utility + functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. + +The binary distribution of this product bundles binaries of +Gson 2.2.4, +which has the following notices: + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2014 The Netty Project + +The Netty Project 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. + +Also, please refer to each LICENSE..txt file, which is located in +the 'license' directory of the distribution file, for the license terms of the +components that this product depends on. + +------------------------------------------------------------------------------- +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified portion of 'Webbit', an event based +WebSocket and HTTP server, which can be obtained at: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +This product contains a modified portion of 'SLF4J', a simple logging +facade for Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +This product contains a modified portion of 'ArrayDeque', written by Josh +Bloch of Google, Inc: + + * LICENSE: + * license/LICENSE.deque.txt (Public Domain) + +This product contains a modified portion of 'Apache Harmony', an open source +Java SE, which can be obtained at: + + * LICENSE: + * license/LICENSE.harmony.txt (Apache License 2.0) + * HOMEPAGE: + * http://archive.apache.org/dist/harmony/ + +This product contains a modified version of Roland Kuhn's ASL2 +AbstractNodeQueue, which is based on Dmitriy Vyukov's non-intrusive MPSC queue. +It can be obtained at: + + * LICENSE: + * license/LICENSE.abstractnodequeue.txt (Public Domain) + * HOMEPAGE: + * https://github.com/akka/akka/blob/wip-2.2.3-for-scala-2.11/akka-actor/src/main/java/akka/dispatch/AbstractNodeQueue.java + +This product contains a modified portion of 'jbzip2', a Java bzip2 compression +and decompression library written by Matthew J. Francis. It can be obtained at: + + * LICENSE: + * license/LICENSE.jbzip2.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jbzip2/ + +This product contains a modified portion of 'libdivsufsort', a C API library to construct +the suffix array and the Burrows-Wheeler transformed string for any input string of +a constant-size alphabet written by Yuta Mori. It can be obtained at: + + * LICENSE: + * license/LICENSE.libdivsufsort.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/libdivsufsort/ + +This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM, + which can be obtained at: + + * LICENSE: + * license/LICENSE.jctools.txt (ASL2 License) + * HOMEPAGE: + * https://github.com/JCTools/JCTools + +This product optionally depends on 'JZlib', a re-implementation of zlib in +pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product optionally depends on 'Compress-LZF', a Java library for encoding and +decoding data in LZF format, written by Tatu Saloranta. It can be obtained at: + + * LICENSE: + * license/LICENSE.compress-lzf.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/ning/compress + +This product optionally depends on 'lz4', a LZ4 Java compression +and decompression library written by Adrien Grand. It can be obtained at: + + * LICENSE: + * license/LICENSE.lz4.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jpountz/lz4-java + +This product optionally depends on 'lzma-java', a LZMA Java compression +and decompression library, which can be obtained at: + + * LICENSE: + * license/LICENSE.lzma-java.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jponge/lzma-java + +This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression +and decompression library written by William Kinney. It can be obtained at: + + * LICENSE: + * license/LICENSE.jfastlz.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jfastlz/ + +This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/protobuf/ + +This product optionally depends on 'Bouncy Castle Crypto APIs' to generate +a temporary self-signed X.509 certificate when the JVM does not provide the +equivalent functionality. It can be obtained at: + + * LICENSE: + * license/LICENSE.bouncycastle.txt (MIT License) + * HOMEPAGE: + * http://www.bouncycastle.org/ + +This product optionally depends on 'Snappy', a compression library produced +by Google Inc, which can be obtained at: + + * LICENSE: + * license/LICENSE.snappy.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/snappy/ + +This product optionally depends on 'JBoss Marshalling', an alternative Java +serialization API, which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://www.jboss.org/jbossmarshalling + +This product optionally depends on 'Caliper', Google's micro- +benchmarking framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.caliper.txt (Apache License 2.0) + * HOMEPAGE: + * http://code.google.com/p/caliper/ + +This product optionally depends on 'Apache Commons Logging', a logging +framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + +This product optionally depends on 'Apache Log4J', a logging framework, which +can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + +This product optionally depends on 'Aalto XML', an ultra-high performance +non-blocking XML processor, which can be obtained at: + + * LICENSE: + * license/LICENSE.aalto-xml.txt (Apache License 2.0) + * HOMEPAGE: + * http://wiki.fasterxml.com/AaltoHome + +This product contains a modified version of 'HPACK', a Java implementation of +the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at: + + * LICENSE: + * license/LICENSE.hpack.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/twitter/hpack + +This product contains a modified portion of 'Apache Commons Lang', a Java library +provides utilities for the java.lang API, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-lang.txt (Apache License 2.0) + * HOMEPAGE: + * https://commons.apache.org/proper/commons-lang/ + +This product contains a modified portion of 'JDOM 1.1', which can be obtained at: + + * LICENSE: + * https://github.com/hunterhacker/jdom/blob/jdom-1.1/core/LICENSE.txt + * HOMEPAGE: + * http://www.jdom.org/ + +The binary distribution of this product bundles binaries of +Commons Codec 1.4, +which has the following notices: + * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + =============================================================================== + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +The binary distribution of this product bundles binaries of +Commons Lang 2.6, +which has the following notices: + * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +The binary distribution of this product bundles binaries of +Apache Log4j 1.2.17, +which has the following notices: + * ResolverUtil.java + Copyright 2005-2006 Tim Fennell + Dumbster SMTP test server + Copyright 2004 Jason Paul Kitchen + TypeUtil.java + Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams + +The binary distribution of this product bundles binaries of +"Java Concurrency in Practice" book annotations 1.0, +which has the following notices: + * Copyright (c) 2005 Brian Goetz and Tim Peierls Released under the Creative + Commons Attribution License (http://creativecommons.org/licenses/by/2.5) + Official home: http://www.jcip.net Any republication or derived work + distributed in source code form must include this copyright and license + notice. + +The binary distribution of this product bundles binaries of +Jetty :: Http Utility 9.3.19., +Jetty :: IO Utility 9.3.19., +Jetty :: Security 9.3.19., +Jetty :: Server Core 9.3.19., +Jetty :: Servlet Handling 9.3.19., +Jetty :: Utilities 9.3.19., +Jetty :: Utilities :: Ajax, +Jetty :: Webapp Application Support 9.3.19., +Jetty :: XML utilities 9.3.19., +which has the following notices: + * ============================================================== + Jetty Web Container + Copyright 1995-2016 Mort Bay Consulting Pty Ltd. + ============================================================== + + The Jetty Web Container is Copyright Mort Bay Consulting Pty Ltd + unless otherwise noted. + + Jetty is dual licensed under both + + * The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html + + and + + * The Eclipse Public 1.0 License + http://www.eclipse.org/legal/epl-v10.html + + Jetty may be distributed under either license. + + ------ + Eclipse + + The following artifacts are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + + The following artifacts are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + + The following artifacts are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + + ------ + Oracle + + The following artifacts are CDDL + GPLv2 with classpath exception. + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * javax.servlet:javax.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + + ------ + Oracle OpenJDK + + If ALPN is used to negotiate HTTP/2 connections, then the following + artifacts may be included in the distribution or downloaded when ALPN + module is selected. + + * java.sun.security.ssl + + These artifacts replace/modify OpenJDK classes. The modififications + are hosted at github and both modified and original are under GPL v2 with + classpath exceptions. + http://openjdk.java.net/legal/gplv2+ce.html + + ------ + OW2 + + The following artifacts are licensed by the OW2 Foundation according to the + terms of http://asm.ow2.org/license.html + + org.ow2.asm:asm-commons + org.ow2.asm:asm + + ------ + Apache + + The following artifacts are ASL2 licensed. + + org.apache.taglibs:taglibs-standard-spec + org.apache.taglibs:taglibs-standard-impl + + ------ + MortBay + + The following artifacts are ASL2 licensed. Based on selected classes from + following Apache Tomcat jars, all ASL2 licensed. + + org.mortbay.jasper:apache-jsp + org.apache.tomcat:tomcat-jasper + org.apache.tomcat:tomcat-juli + org.apache.tomcat:tomcat-jsp-api + org.apache.tomcat:tomcat-el-api + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-api + org.apache.tomcat:tomcat-util-scan + org.apache.tomcat:tomcat-util + + org.mortbay.jasper:apache-el + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-el-api + + ------ + Mortbay + + The following artifacts are CDDL + GPLv2 with classpath exception. + + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + org.eclipse.jetty.toolchain:jetty-schemas + + ------ + Assorted + + The UnixCrypt.java code implements the one way cryptography used by + Unix systems for simple password protection. Copyright 1996 Aki Yoshida, + modified April 2001 by Iris Van den Broeke, Daniel Deville. + Permission to use, copy, modify and distribute UnixCrypt + for non-commercial or commercial purposes and without fee is + granted provided that the copyright notice appears in all copies./ + +The binary distribution of this product bundles binaries of +Snappy for Java 1.0.4.1, +which has the following notices: + * This product includes software developed by Google + Snappy: http://code.google.com/p/snappy/ (New BSD License) + + This product includes software developed by Apache + PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ + (Apache 2.0 license) + + This library containd statically linked libstdc++. This inclusion is allowed by + "GCC RUntime Library Exception" + http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html + + == Contributors == + * Tatu Saloranta + * Providing benchmark suite + * Alec Wysoker + * Performance and memory usage improvement + +The binary distribution of this product bundles binaries of +Xerces2 Java Parser 2.9.1, +which has the following notices: + * ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the Apache Xerces Java distribution. == + ========================================================================= + + Apache Xerces Java + Copyright 1999-2007 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of this software were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - voluntary contributions made by Paul Eng on behalf of the + Apache Software Foundation that were originally developed at iClick, Inc., + software copyright (c) 1999. + +The binary distribution of this product bundles binaries of +Logback Classic Module 1.1.2, +Logback Core Module 1.1.2, +which has the following notices: + * Logback: the reliable, generic, fast and flexible logging framework. + Copyright (C) 1999-2012, QOS.ch. All rights reserved. + +The binary distribution of this product bundles binaries of +Apache HBase - Annotations 1.2.6, +Apache HBase - Client 1.2.6, +Apache HBase - Common 1.2.6, +Apache HBase - Hadoop Compatibility 1.2.6, +Apache HBase - Hadoop Two Compatibility 1.2.6, +Apache HBase - Prefix Tree 1.2.6, +Apache HBase - Procedure 1.2.6, +Apache HBase - Protocol 1.2.6, +Apache HBase - Server 1.2.6, +which has the following notices: + * Apache HBase + Copyright 2007-2015 The Apache Software Foundation + + -- + This product incorporates portions of the 'Hadoop' project + + Copyright 2007-2009 The Apache Software Foundation + + Licensed under the Apache License v2.0 + -- + Our Orca logo we got here: http://www.vectorfree.com/jumping-orca + It is licensed Creative Commons Attribution 3.0. + See https://creativecommons.org/licenses/by/3.0/us/ + We changed the logo by stripping the colored background, inverting + it and then rotating it some. + + Later we found that vectorfree.com image is not properly licensed. + The original is owned by vectorportal.com. The original was + relicensed so we could use it as Creative Commons Attribution 3.0. + The license is bundled with the download available here: + http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp + -- + This product includes portions of the Bootstrap project v3.0.0 + + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License v2.0 + + This product uses the Glyphicons Halflings icon set. + + http://glyphicons.com/ + + Copyright Jan Kovařík + + Licensed under the Apache License v2.0 as a part of the Bootstrap project. + + -- + This product includes portions of the Guava project v14, specifically + 'hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java' + + Copyright (C) 2007 The Guava Authors + + Licensed under the Apache License, Version 2.0 + +The binary distribution of this product bundles binaries of +Phoenix Core 4.7.0, +which has the following notices: + Apache Phoenix + Copyright 2013-2016 The Apache Software Foundation + + This product includes software developed by The Apache Software + Foundation (http://www.apache.org/). + + This also includes: + + The phoenix-spark module has been adapted from the phoenix-spark library + distributed under the terms of the Apache 2 license. Original source copyright: + Copyright 2014 Simply Measured, Inc. + Copyright 2015 Interset Software Inc. + + The file bin/daemon.py is based on the file of the same name in python-daemon 2.0.5 + (https://pypi.python.org/pypi/python-daemon/). Original source copyright: + # Copyright © 2008–2015 Ben Finney + # Copyright © 2007–2008 Robert Niederreiter, Jens Klein + # Copyright © 2004–2005 Chad J. Schroeder + # Copyright © 2003 Clark Evans + # Copyright © 2002 Noah Spurrier + # Copyright © 2001 Jürgen Hermann + +The binary distribution of this product bundles binaries of +Plexus Cipher: encryption/decryption Component 1.4, +which has the following notices: + * The code in this component contains a class - Base64 taken from http://juliusdavies.ca/svn/not-yet-commons-ssl/tags/commons-ssl-0.3.10/src/java/org/apache/commons/ssl/Base64.java + which is Apache license: http://www.apache.org/licenses/LICENSE-2.0 + + The PBE key processing routine PBECipher.createCipher() is adopted from http://juliusdavies.ca/svn/not-yet-commons-ssl/tags/commons-ssl-0.3.10/src/java/org/apache/commons/ssl/OpenSSL.java + which is also Apache APL-2.0 license: http://www.apache.org/licenses/LICENSE-2.0 + +The binary distribution of this product bundles binaries of +software.amazon.ion:ion-java 1.0.1, +which has the following notices: + * Amazon Ion Java Copyright 2007-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + +The binary distribution of this product bundles binaries of +joda-time:joda-time:2.9.9 +which has the following notices: + * ============================================================================= + = NOTICE file corresponding to section 4d of the Apache License Version 2.0 = + ============================================================================= + This product includes software developed by + Joda.org (http://www.joda.org/). + +The binary distribution of this product bundles binaries of +Ehcache 3.3.1, +which has the following notices: + * Ehcache V3 Copyright 2014-2016 Terracotta, Inc. + +The binary distribution of this product bundles binaries of +snakeyaml (https://bitbucket.org/asomov/snakeyaml), +which has the following notices: + * Copyright (c) 2008, http://www.snakeyaml.org + +The binary distribution of this product bundles binaries of +swagger-annotations (https://github.com/swagger-api/swagger-core), +which has the following notices: + * Copyright 2016 SmartBear Software + +The binary distribution of this product bundles binaries of +metrics-core 3.2.4 +which has the following notices: + * Copyright 2010-2013 Coda Hale and Yammer, Inc. + + This product includes software developed by Coda Hale and Yammer, Inc. + + This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, + LongAdder), which was released with the following comments: + + Written by Doug Lea with assistance from members of JCP JSR-166 + Expert Group and released to the public domain, as explained at + http://creativecommons.org/publicdomain/zero/1.0/ + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +Apache Commons Collections +Copyright 2001-2015 The Apache Software Foundation + +Apache Commons Logging +Copyright 2003-2013 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons BeanUtils +Copyright 2000-2016 The Apache Software Foundation + +Apache Commons Configuration +Copyright 2001-2017 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2014 The Apache Software Foundation + +This product includes software from the Spring Framework, +under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +htrace-core4 +Copyright 2016 The Apache Software Foundation + +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +force-shading +Copyright 2018 The Apache Software Foundation + +flink-hadoop-fs +Copyright 2014-2018 The Apache Software Foundation + +Apache HttpClient +Copyright 1999-2017 The Apache Software Foundation + +Apache HttpCore +Copyright 2005-2017 The Apache Software Foundation + +Apache Commons Codec +Copyright 2002-2014 The Apache Software Foundation + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + +=============================================================================== + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +============================================================================= += NOTICE file corresponding to section 4d of the Apache License Version 2.0 = +============================================================================= +This product includes software developed by +Joda.org (http://www.joda.org/). + +flink-metrics-prometheus +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- io.prometheus:simpleclient:0.3.0 +- io.prometheus:simpleclient_common:0.3.0 +- io.prometheus:simpleclient_httpserver:0.3.0 +- io.prometheus:simpleclient_pushgateway:0.3.0 + +flink-s3-fs-base +Copyright 2014-2018 The Apache Software Foundation + +This project includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hadoop:hadoop-aws:3.1.0 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpclient:4.5.3 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- com.amazonaws:aws-java-sdk-core:1.11.271 +- com.amazonaws:aws-java-sdk-dynamodb:1.11.271 +- com.amazonaws:aws-java-sdk-kms:1.11.271 +- com.amazonaws:aws-java-sdk-s3:1.11.271 +- com.amazonaws:jmespath-java:1.11.271 +- software.amazon.ion:ion-java:1.0.2 +- com.fasterxml.jackson.core:jackson-annotations:2.6.0 +- com.fasterxml.jackson.core:jackson-core:2.6.7 +- com.fasterxml.jackson.core:jackson-databind:2.6.7.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.6.7 +- joda-time:joda-time:2.5 + +flink-fs-hadoop-shaded +Copyright 2014-2018 The Apache Software Foundation + +- org.apache.hadoop:hadoop-annotations:3.1.0 +- org.apache.hadoop:hadoop-auth:3.1.0 +- org.apache.hadoop:hadoop-common:3.1.0 +- org.apache.htrace:htrace-core4:4.1.0-incubating +- org.apache.commons:commons-configuration2:2.1.1 +- org.apache.commons:commons-lang3:3.3.2 +- commons-lang:commons-lang:2.6 +- commons-collections:commons-collections:3.2.2 +- commons-io:commons-io:2.4 +- commons-logging:commons-logging:1.1.3 +- commons-beanutils:commons-beanutils:1.9.3 +- com.google.guava:guava:11.0.2 +- com.fasterxml.jackson.core:jackson-annotations:2.7.0 +- com.fasterxml.jackson.core:jackson-core:2.7.8 +- com.fasterxml.jackson.core:jackson-databind:2.7.8 +- com.fasterxml.woodstox:woodstox-core:5.0.3 + +This project bundles the following dependencies under the Go License (https://golang.org/LICENSE). +See bundled license files for details. + +- com.google.re2j:re2j:1.1 + +This project bundles the following dependencies under BSD License (https://opensource.org/licenses/bsd-license.php). +See bundled license files for details. + +- org.codehaus.woodstox:stax2-api:3.1.4 (https://github.com/FasterXML/stax2-api/tree/stax2-api-3.1.4) + +flink-sql-client +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- org.jline:jline-terminal:3.9.0 +- org.jline:jline-reader:3.9.0 + + +flink-table +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.esri.geometry:esri-geometry-api:2.0.0 +- com.google.guava:guava:19.0 +- joda-time:joda-time:2.5 +- net.hydromatic:aggdesigner-algorithm:6.0 +- org.apache.calcite:calcite-core:1.17.0 +- org.apache.calcite:calcite-linq4j:1.17.0 +- org.apache.calcite.avatica:avatica-core:1.12.0 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details + +- org.codehaus.janino:janino:3.0.7 +- org.codehaus.janino:commons-compiler:3.0.7 + +flink-table-common +Copyright 2014-2018 The Apache Software Foundation + +Calcite Core +Copyright 2012-2018 The Apache Software Foundation + +Apache Calcite Avatica +Copyright 2012-2018 The Apache Software Foundation + +Calcite Linq4j +Copyright 2012-2018 The Apache Software Foundation + +============================================================================= += NOTICE file corresponding to section 4d of the Apache License Version 2.0 = +============================================================================= +This product includes software developed by +Joda.org (http://www.joda.org/). + +force-shading +Copyright 2018 The Apache Software Foundation + +flink-cep +Copyright 2014-2018 The Apache Software Foundation + +flink-streaming-python +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Python Software Foundation License. (https://opensource.org/licenses/PythonSoftFoundation.php) +See bundled license files for details. + +- org.python:jython-standalone:2.7.0 + +flink-metrics-graphite +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- io.dropwizard.metrics:metrics-core:3.1.5 +- io.dropwizard.metrics:metrics-graphite:3.1.5 diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 89228b39271f8..8627f1a56a18a 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -58,9 +58,27 @@ under the License. conf 0644 + + + ../LICENSE + + 0644 + + + + ../NOTICE-binary + + NOTICE + 0644 + + + ../licenses-binary + licenses + 0644 + src/main/flink-bin/bin @@ -130,17 +148,6 @@ under the License.
    - - - ../ - - 0644 - - LICENSE* - NOTICE* - - - diff --git a/licenses-binary/LICENSE-hdrhistogram b/licenses-binary/LICENSE-hdrhistogram new file mode 100644 index 0000000000000..09c38ea51608e --- /dev/null +++ b/licenses-binary/LICENSE-hdrhistogram @@ -0,0 +1,125 @@ +The code was Written by Gil Tene, Michael Barker, and Matt Warren, +and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ + +Creative Commons Legal Code + +CC0 1.0 Universal + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS + PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM + THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED + HEREUNDER. + +Statement of Purpose + +The laws of most jurisdictions throughout the world automatically confer +exclusive Copyright and Related Rights (defined below) upon the creator +and subsequent owner(s) (each and all, an "owner") of an original work of +authorship and/or a database (each, a "Work"). + +Certain owners wish to permanently relinquish those rights to a Work for +the purpose of contributing to a commons of creative, cultural and +scientific works ("Commons") that the public can reliably and without fear +of later claims of infringement build upon, modify, incorporate in other +works, reuse and redistribute as freely as possible in any form whatsoever +and for any purposes, including without limitation commercial purposes. +These owners may contribute to the Commons to promote the ideal of a free +culture and the further production of creative, cultural and scientific +works, or to gain reputation or greater distribution for their Work in +part through the use and efforts of others. + +For these and/or other purposes and motivations, and without any +expectation of additional consideration or compensation, the person +associating CC0 with a Work (the "Affirmer"), to the extent that he or she +is an owner of Copyright and Related Rights in the Work, voluntarily +elects to apply CC0 to the Work and publicly distribute the Work under its +terms, with knowledge of his or her Copyright and Related Rights in the +Work and the meaning and intended legal effect of CC0 on those rights. + +1. Copyright and Related Rights. A Work made available under CC0 may be +protected by copyright and related or neighboring rights ("Copyright and +Related Rights"). Copyright and Related Rights include, but are not +limited to, the following: + + i. the right to reproduce, adapt, distribute, perform, display, + communicate, and translate a Work; + ii. moral rights retained by the original author(s) and/or performer(s); +iii. publicity and privacy rights pertaining to a person's image or + likeness depicted in a Work; + iv. rights protecting against unfair competition in regards to a Work, + subject to the limitations in paragraph 4(a), below; + v. rights protecting the extraction, dissemination, use and reuse of data + in a Work; + vi. database rights (such as those arising under Directive 96/9/EC of the + European Parliament and of the Council of 11 March 1996 on the legal + protection of databases, and under any national implementation + thereof, including any amended or successor version of such + directive); and +vii. other similar, equivalent or corresponding rights throughout the + world based on applicable law or treaty, and any national + implementations thereof. + +2. Waiver. To the greatest extent permitted by, but not in contravention +of, applicable law, Affirmer hereby overtly, fully, permanently, +irrevocably and unconditionally waives, abandons, and surrenders all of +Affirmer's Copyright and Related Rights and associated claims and causes +of action, whether now known or unknown (including existing as well as +future claims and causes of action), in the Work (i) in all territories +worldwide, (ii) for the maximum duration provided by applicable law or +treaty (including future time extensions), (iii) in any current or future +medium and for any number of copies, and (iv) for any purpose whatsoever, +including without limitation commercial, advertising or promotional +purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each +member of the public at large and to the detriment of Affirmer's heirs and +successors, fully intending that such Waiver shall not be subject to +revocation, rescission, cancellation, termination, or any other legal or +equitable action to disrupt the quiet enjoyment of the Work by the public +as contemplated by Affirmer's express Statement of Purpose. + +3. Public License Fallback. Should any part of the Waiver for any reason +be judged legally invalid or ineffective under applicable law, then the +Waiver shall be preserved to the maximum extent permitted taking into +account Affirmer's express Statement of Purpose. In addition, to the +extent the Waiver is so judged Affirmer hereby grants to each affected +person a royalty-free, non transferable, non sublicensable, non exclusive, +irrevocable and unconditional license to exercise Affirmer's Copyright and +Related Rights in the Work (i) in all territories worldwide, (ii) for the +maximum duration provided by applicable law or treaty (including future +time extensions), (iii) in any current or future medium and for any number +of copies, and (iv) for any purpose whatsoever, including without +limitation commercial, advertising or promotional purposes (the +"License"). The License shall be deemed effective as of the date CC0 was +applied by Affirmer to the Work. Should any part of the License for any +reason be judged legally invalid or ineffective under applicable law, such +partial invalidity or ineffectiveness shall not invalidate the remainder +of the License, and in such case Affirmer hereby affirms that he or she +will not (i) exercise any of his or her remaining Copyright and Related +Rights in the Work or (ii) assert any associated claims and causes of +action with respect to the Work, in either case contrary to Affirmer's +express Statement of Purpose. + +4. Limitations and Disclaimers. + + a. No trademark or patent rights held by Affirmer are waived, abandoned, + surrendered, licensed or otherwise affected by this document. + b. Affirmer offers the Work as-is and makes no representations or + warranties of any kind concerning the Work, express, implied, + statutory or otherwise, including without limitation warranties of + title, merchantability, fitness for a particular purpose, non + infringement, or the absence of latent or other defects, accuracy, or + the present or absence of errors, whether or not discoverable, all to + the greatest extent permissible under applicable law. + c. Affirmer disclaims responsibility for clearing rights of other persons + that may apply to the Work or any use thereof, including without + limitation any person's Copyright and Related Rights in the Work. + Further, Affirmer disclaims responsibility for obtaining any necessary + consents, permissions or other rights required for any use of the + Work. + d. Affirmer understands and acknowledges that Creative Commons is not a + party to this document and has no duty or obligation with respect to + this CC0 or use of the Work. diff --git a/licenses-binary/LICENSE-protobuf b/licenses-binary/LICENSE-protobuf new file mode 100644 index 0000000000000..21645bec9d5d1 --- /dev/null +++ b/licenses-binary/LICENSE-protobuf @@ -0,0 +1,36 @@ +Copyright 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + diff --git a/licenses-binary/LICENSE-re2j b/licenses-binary/LICENSE-re2j new file mode 100644 index 0000000000000..b620ae68fe332 --- /dev/null +++ b/licenses-binary/LICENSE-re2j @@ -0,0 +1,32 @@ +This is a work derived from Russ Cox's RE2 in Go, whose license +http://golang.org/LICENSE is as follows: + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE-stax2api b/licenses-binary/LICENSE-stax2api new file mode 100644 index 0000000000000..0ed6361699658 --- /dev/null +++ b/licenses-binary/LICENSE-stax2api @@ -0,0 +1,22 @@ +Copyright woodstox stax2api contributors. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, +INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, +BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, +WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE-xmlenc b/licenses-binary/LICENSE-xmlenc new file mode 100644 index 0000000000000..a1256bdf6f8fd --- /dev/null +++ b/licenses-binary/LICENSE-xmlenc @@ -0,0 +1,28 @@ +Copyright 2003-2005, Ernst de Haan +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/licenses-binary/LICENSE.angular b/licenses-binary/LICENSE.angular new file mode 100644 index 0000000000000..91f0644936811 --- /dev/null +++ b/licenses-binary/LICENSE.angular @@ -0,0 +1,22 @@ +The MIT License + +Copyright (c) 2010-2018 Google, Inc. http://angularjs.org + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + diff --git a/licenses-binary/LICENSE.angular-drag-and-drop-list b/licenses-binary/LICENSE.angular-drag-and-drop-list new file mode 100644 index 0000000000000..1787fc65befe1 --- /dev/null +++ b/licenses-binary/LICENSE.angular-drag-and-drop-list @@ -0,0 +1,22 @@ +The MIT License (MIT) + +Copyright (c) 2014 Marcel Juenemann +Copyright (c) 2014-2016 Google Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/licenses-binary/LICENSE.angular-moment b/licenses-binary/LICENSE.angular-moment new file mode 100644 index 0000000000000..fe9db79b9113a --- /dev/null +++ b/licenses-binary/LICENSE.angular-moment @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2013-2016 Uri Shaked and contributors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses-binary/LICENSE.angular-ui-router b/licenses-binary/LICENSE.angular-ui-router new file mode 100644 index 0000000000000..6413b092d70f7 --- /dev/null +++ b/licenses-binary/LICENSE.angular-ui-router @@ -0,0 +1,21 @@ +The MIT License + +Copyright (c) 2013-2015 The AngularUI Team, Karsten Sperling + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses-binary/LICENSE.arpack_combined_all b/licenses-binary/LICENSE.arpack_combined_all new file mode 100644 index 0000000000000..6a6cdbb5100e3 --- /dev/null +++ b/licenses-binary/LICENSE.arpack_combined_all @@ -0,0 +1,8 @@ +Copyright © 2018 The University of Tennessee. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: +· Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. +· Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer listed in this license in the documentation and/or other materials provided with the distribution. +· Neither the name of the copyright holders nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +This software is provided by the copyright holders and contributors "as is" and any express or implied warranties, including, but not limited to, the implied warranties of merchantability and fitness for a particular purpose are disclaimed. in no event shall the copyright owner or contributors be liable for any direct, indirect, incidental, special, exemplary, or consequential damages (including, but not limited to, procurement of substitute goods or services; loss of use, data, or profits; or business interruption) however caused and on any theory of liability, whether in contract, strict liability, or tort (including negligence or otherwise) arising in any way out of the use of this software, even if advised of the possibility of such damage. diff --git a/licenses-binary/LICENSE.asm.txt b/licenses-binary/LICENSE.asm.txt new file mode 100644 index 0000000000000..62ffbccb6837a --- /dev/null +++ b/licenses-binary/LICENSE.asm.txt @@ -0,0 +1,31 @@ +ASM: a very small and fast Java bytecode manipulation framework + +Copyright (c) 2000-2011 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/licenses-binary/LICENSE.base64 b/licenses-binary/LICENSE.base64 new file mode 100644 index 0000000000000..31ebc840539c1 --- /dev/null +++ b/licenses-binary/LICENSE.base64 @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuate of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/licenses-binary/LICENSE.bootstrap b/licenses-binary/LICENSE.bootstrap new file mode 100644 index 0000000000000..6ca0ceecc5d79 --- /dev/null +++ b/licenses-binary/LICENSE.bootstrap @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2011-2018 Twitter, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses-binary/LICENSE.cddlv1.0 b/licenses-binary/LICENSE.cddlv1.0 new file mode 100644 index 0000000000000..468e7516a87ff --- /dev/null +++ b/licenses-binary/LICENSE.cddlv1.0 @@ -0,0 +1,129 @@ +1. Definitions. + +1.1. "Contributor" means each individual or entity that creates or contributes to the creation of Modifications. + +1.2. "Contributor Version" means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + +1.3. "Covered Software" means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + +1.4. "Executable" means the Covered Software in any form other than Source Code. + +1.5. "Initial Developer" means the individual or entity that first makes Original Software available under this License. + +1.6. "Larger Work" means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + +1.7. "License" means this document. + +1.8. "Licensable" means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + +1.9. "Modifications" means the Source Code and Executable form of any of the following: + +A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + +B. Any new file that contains any part of the Original Software or previous Modification; or + +C. Any new file that is contributed or otherwise made available under the terms of this License. + +1.10. "Original Software" means the Source Code and Executable form of computer software code that is originally released under this License. + +1.11. "Patent Claims" means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + +1.12. "Source Code" means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + +1.13. "You" (or "Your") means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, "You" includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, "control" means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + +2.1. The Initial Developer Grant. + +Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + +(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + +(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + +(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + +(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + +2.2. Contributor Grant. + +Conditioned upon Your compliance with Section 3.1 below and +subject to third party intellectual property claims, each +Contributor hereby grants You a world-wide, royalty-free, +non-exclusive license: + +(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + +(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + +(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. +(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + +3.1. Availability of Source Code. + +Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + +3.2. Modifications. + +The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + +3.3. Required Notices. + +You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + +3.4. Application of Additional Terms. + +You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + +3.5. Distribution of Executable Versions. + +You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + +3.6. Larger Works. + +You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + +4.1. New Versions. + +Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + +4.2. Effect of New Versions. + +You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + +4.3. Modified Versions. + +When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + +6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + +6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as "Participant") alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + +6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a "commercial item," as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer software" (as that term is defined at 48 C.F.R. ¤ 252.227-7014(a)(1)) and "commercial computer software documentation" as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. + +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. diff --git a/licenses-binary/LICENSE.cddlv1.1 b/licenses-binary/LICENSE.cddlv1.1 new file mode 100644 index 0000000000000..75372e3d2c9a4 --- /dev/null +++ b/licenses-binary/LICENSE.cddlv1.1 @@ -0,0 +1,348 @@ +1. Definitions. + + 1.1. "Contributor" means each individual or entity that creates or + contributes to the creation of Modifications. + + 1.2. "Contributor Version" means the combination of the Original + Software, prior Modifications used by a Contributor (if any), and + the Modifications made by that particular Contributor. + + 1.3. "Covered Software" means (a) the Original Software, or (b) + Modifications, or (c) the combination of files containing Original + Software with files containing Modifications, in each case including + portions thereof. + + 1.4. "Executable" means the Covered Software in any form other than + Source Code. + + 1.5. "Initial Developer" means the individual or entity that first + makes Original Software available under this License. + + 1.6. "Larger Work" means a work which combines Covered Software or + portions thereof with code not governed by the terms of this License. + + 1.7. "License" means this document. + + 1.8. "Licensable" means having the right to grant, to the maximum + extent possible, whether at the time of the initial grant or + subsequently acquired, any and all of the rights conveyed herein. + + 1.9. "Modifications" means the Source Code and Executable form of + any of the following: + + A. Any file that results from an addition to, deletion from or + modification of the contents of a file containing Original Software + or previous Modifications; + + B. Any new file that contains any part of the Original Software or + previous Modification; or + + C. Any new file that is contributed or otherwise made available + under the terms of this License. + + 1.10. "Original Software" means the Source Code and Executable form + of computer software code that is originally released under this + License. + + 1.11. "Patent Claims" means any patent claim(s), now owned or + hereafter acquired, including without limitation, method, process, + and apparatus claims, in any patent Licensable by grantor. + + 1.12. "Source Code" means (a) the common form of computer software + code in which modifications are made and (b) associated + documentation included in or with such code. + + 1.13. "You" (or "Your") means an individual or a legal entity + exercising rights under, and complying with all of the terms of, + this License. For legal entities, "You" includes any entity which + controls, is controlled by, or is under common control with You. For + purposes of this definition, "control" means (a) the power, direct + or indirect, to cause the direction or management of such entity, + whether by contract or otherwise, or (b) ownership of more than + fifty percent (50%) of the outstanding shares or beneficial + ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, the Initial Developer + hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Initial Developer, to use, reproduce, + modify, display, perform, sublicense and distribute the Original + Software (or portions thereof), with or without Modifications, + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of + Original Software, to make, have made, use, practice, sell, and + offer for sale, and/or otherwise dispose of the Original Software + (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on + the date Initial Developer first distributes or otherwise makes the + Original Software available to a third party under the terms of this + License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is + granted: (1) for code that You delete from the Original Software, or + (2) for infringements caused by: (i) the modification of the + Original Software, or (ii) the combination of the Original Software + with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, each Contributor hereby + grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Contributor to use, reproduce, modify, + display, perform, sublicense and distribute the Modifications + created by such Contributor (or portions thereof), either on an + unmodified basis, with other Modifications, as Covered Software + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling + of Modifications made by that Contributor either alone and/or in + combination with its Contributor Version (or portions of such + combination), to make, use, sell, offer for sale, have made, and/or + otherwise dispose of: (1) Modifications made by that Contributor (or + portions thereof); and (2) the combination of Modifications made by + that Contributor with its Contributor Version (or portions of such + combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective + on the date Contributor first distributes or otherwise makes the + Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is + granted: (1) for any code that Contributor has deleted from the + Contributor Version; (2) for infringements caused by: (i) third + party modifications of Contributor Version, or (ii) the combination + of Modifications made by that Contributor with other software + (except as part of the Contributor Version) or other devices; or (3) + under Patent Claims infringed by Covered Software in the absence of + Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available + in Executable form must also be made available in Source Code form + and that Source Code form must be distributed only under the terms + of this License. You must include a copy of this License with every + copy of the Source Code form of the Covered Software You distribute + or otherwise make available. You must inform recipients of any such + Covered Software in Executable form as to how they can obtain such + Covered Software in Source Code form in a reasonable manner on or + through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are + governed by the terms of this License. You represent that You + believe Your Modifications are Your original creation(s) and/or You + have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that + identifies You as the Contributor of the Modification. You may not + remove or alter any copyright, patent or trademark notices contained + within the Covered Software, or any notices of licensing or any + descriptive text giving attribution to any Contributor or the + Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in + Source Code form that alters or restricts the applicable version of + this License or the recipients' rights hereunder. You may choose to + offer, and to charge a fee for, warranty, support, indemnity or + liability obligations to one or more recipients of Covered Software. + However, you may do so only on Your own behalf, and not on behalf of + the Initial Developer or any Contributor. You must make it + absolutely clear that any such warranty, support, indemnity or + liability obligation is offered by You alone, and You hereby agree + to indemnify the Initial Developer and every Contributor for any + liability incurred by the Initial Developer or such Contributor as a + result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under + the terms of this License or under the terms of a license of Your + choice, which may contain terms different from this License, + provided that You are in compliance with the terms of this License + and that the license for the Executable form does not attempt to + limit or alter the recipient's rights in the Source Code form from + the rights set forth in this License. If You distribute the Covered + Software in Executable form under a different license, You must make + it absolutely clear that any terms which differ from this License + are offered by You alone, not by the Initial Developer or + Contributor. You hereby agree to indemnify the Initial Developer and + every Contributor for any liability incurred by the Initial + Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with + other code not governed by the terms of this License and distribute + the Larger Work as a single product. In such a case, You must make + sure the requirements of this License are fulfilled for the Covered + Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or + new versions of this License from time to time. Each version will be + given a distinguishing version number. Except as provided in Section + 4.3, no one other than the license steward has the right to modify + this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the + Covered Software available under the terms of the version of the + License under which You originally received the Covered Software. If + the Initial Developer includes a notice in the Original Software + prohibiting it from being distributed or otherwise made available + under any subsequent version of the License, You must distribute and + make the Covered Software available under the terms of the version + of the License under which You originally received the Covered + Software. Otherwise, You may also choose to use, distribute or + otherwise make the Covered Software available under the terms of any + subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new + license for Your Original Software, You may create and use a + modified version of this License if You: (a) rename the license and + remove any references to the name of the license steward (except to + note that the license differs from this License); and (b) otherwise + make it clear that the license contains terms which differ from this + License. + +5. DISCLAIMER OF WARRANTY. + + COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, + WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, + INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE + IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR + NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF + THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE + DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY + OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, + REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN + ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS + AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate + automatically if You fail to comply with terms herein and fail to + cure such breach within 30 days of becoming aware of the breach. + Provisions which, by their nature, must remain in effect beyond the + termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding + declaratory judgment actions) against Initial Developer or a + Contributor (the Initial Developer or Contributor against whom You + assert such claim is referred to as "Participant") alleging that the + Participant Software (meaning the Contributor Version where the + Participant is a Contributor or the Original Software where the + Participant is the Initial Developer) directly or indirectly + infringes any patent, then any and all rights granted directly or + indirectly to You by such Participant, the Initial Developer (if the + Initial Developer is not the Participant) and all Contributors under + Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice + from Participant terminate prospectively and automatically at the + expiration of such 60 day notice period, unless if within such 60 + day period You withdraw Your claim with respect to the Participant + Software against such Participant either unilaterally or pursuant to + a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant + alleging that the Participant Software directly or indirectly + infringes any patent where such claim is resolved (such as by + license or settlement) prior to the initiation of patent + infringement litigation, then the reasonable value of the licenses + granted by such Participant under Sections 2.1 or 2.2 shall be taken + into account in determining the amount or value of any payment or + license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, + all end user licenses that have been validly granted by You or any + distributor hereunder prior to termination (excluding licenses + granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + + UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT + (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE + INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF + COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE + TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR + CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT + LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER + FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR + LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE + POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT + APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH + PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH + LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR + LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION + AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + + The Covered Software is a "commercial item," as that term is defined + in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer + software" (as that term is defined at 48 C.F.R. § + 252.227-7014(a)(1)) and "commercial computer software documentation" + as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent + with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 + (June 1995), all U.S. Government End Users acquire Covered Software + with only those rights set forth herein. This U.S. Government Rights + clause is in lieu of, and supersedes, any other FAR, DFAR, or other + clause or provision that addresses Government rights in computer + software under this License. + +9. MISCELLANEOUS. + + This License represents the complete agreement concerning subject + matter hereof. If any provision of this License is held to be + unenforceable, such provision shall be reformed only to the extent + necessary to make it enforceable. This License shall be governed by + the law of the jurisdiction specified in a notice contained within + the Original Software (except to the extent applicable law, if any, + provides otherwise), excluding such jurisdiction's conflict-of-law + provisions. Any litigation relating to this License shall be subject + to the jurisdiction of the courts located in the jurisdiction and + venue specified in a notice contained within the Original Software, + with the losing party responsible for costs, including, without + limitation, court costs and reasonable attorneys' fees and expenses. + The application of the United Nations Convention on Contracts for + the International Sale of Goods is expressly excluded. Any law or + regulation which provides that the language of a contract shall be + construed against the drafter shall not apply to this License. You + agree that You alone are responsible for compliance with the United + States export administration regulations (and the export control + laws and regulation of any other countries) when You use, distribute + or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + + As between Initial Developer and the Contributors, each party is + responsible for claims and damages arising, directly or indirectly, + out of its utilization of rights under this License and You agree to + work with Initial Developer and Contributors to distribute such + responsibility on an equitable basis. Nothing herein is intended or + shall be deemed to constitute any admission of liability. diff --git a/licenses-binary/LICENSE.core b/licenses-binary/LICENSE.core new file mode 100644 index 0000000000000..b7d28491e4b6b --- /dev/null +++ b/licenses-binary/LICENSE.core @@ -0,0 +1,49 @@ +Copyright (c) 2013 Samuel Halliday +Copyright (c) 1992-2011 The University of Tennessee and The University + of Tennessee Research Foundation. All rights + reserved. +Copyright (c) 2000-2011 The University of California Berkeley. All + rights reserved. +Copyright (c) 2006-2011 The University of Colorado Denver. All rights + reserved. + +$COPYRIGHT$ + +Additional copyrights may follow + +$HEADER$ + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +- Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +- Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer listed + in this license in the documentation and/or other materials + provided with the distribution. + +- Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +The copyright holders provide no reassurances that the source code +provided does not infringe any patent, copyright, or any other +intellectual property rights of third parties. The copyright holders +disclaim any liability to any recipient for claims brought against +recipient by any third party for infringement of that parties +intellectual property rights. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.d3 b/licenses-binary/LICENSE.d3 new file mode 100644 index 0000000000000..1d9d875edb469 --- /dev/null +++ b/licenses-binary/LICENSE.d3 @@ -0,0 +1,27 @@ +Copyright 2010-2017 Mike Bostock +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the author nor the names of contributors may be used to + endorse or promote products derived from this software without specific prior + written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.dagre b/licenses-binary/LICENSE.dagre new file mode 100644 index 0000000000000..e3c8f95557dbb --- /dev/null +++ b/licenses-binary/LICENSE.dagre @@ -0,0 +1,19 @@ +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/licenses-binary/LICENSE.dagre-d3 b/licenses-binary/LICENSE.dagre-d3 new file mode 100644 index 0000000000000..1d64ed68ce64d --- /dev/null +++ b/licenses-binary/LICENSE.dagre-d3 @@ -0,0 +1,19 @@ +Copyright (c) 2013 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses-binary/LICENSE.ev-emitter b/licenses-binary/LICENSE.ev-emitter new file mode 100644 index 0000000000000..3e30b1aeff26a --- /dev/null +++ b/licenses-binary/LICENSE.ev-emitter @@ -0,0 +1,7 @@ +Copyright © 2018 David DeSandro + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/licenses-binary/LICENSE.font-awesome b/licenses-binary/LICENSE.font-awesome new file mode 100644 index 0000000000000..b9fb2c6e6ef06 --- /dev/null +++ b/licenses-binary/LICENSE.font-awesome @@ -0,0 +1,97 @@ +The MIT license + +Copyright + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + + + SIL OPEN FONT LICENSE + +Version 1.1 - 26 February 2007 +PREAMBLE + +The goals of the Open Font License (OFL) are to stimulate worldwide +development of collaborative font projects, to support the font creation +efforts of academic and linguistic communities, and to provide a free and +open framework in which fonts may be shared and improved in partnership +with others. + +The OFL allows the licensed fonts to be used, studied, modified and +redistributed freely as long as they are not sold by themselves. The +fonts, including any derivative works, can be bundled, embedded, +redistributed and/or sold with any software provided that any reserved +names are not used by derivative works. The fonts and derivatives, +however, cannot be released under any other type of license. The +requirement for fonts to remain under this license does not apply +to any document created using the fonts or their derivatives. +DEFINITIONS + +"Font Software" refers to the set of files released by the Copyright +Holder(s) under this license and clearly marked as such. This may +include source files, build scripts and documentation. + +"Reserved Font Name" refers to any names specified as such after the +copyright statement(s). + +"Original Version" refers to the collection of Font Software components as +distributed by the Copyright Holder(s). + +"Modified Version" refers to any derivative made by adding to, deleting, +or substituting — in part or in whole — any of the components of the +Original Version, by changing formats or by porting the Font Software to a +new environment. + +"Author" refers to any designer, engineer, programmer, technical +writer or other person who contributed to the Font Software. +PERMISSION & CONDITIONS + +Permission is hereby granted, free of charge, to any person obtaining +a copy of the Font Software, to use, study, copy, merge, embed, modify, +redistribute, and sell modified and unmodified copies of the Font +Software, subject to the following conditions: + +1) Neither the Font Software nor any of its individual components, +in Original or Modified Versions, may be sold by itself. + +2) Original or Modified Versions of the Font Software may be bundled, +redistributed and/or sold with any software, provided that each copy +contains the above copyright notice and this license. These can be +included either as stand-alone text files, human-readable headers or +in the appropriate machine-readable metadata fields within text or +binary files as long as those fields can be easily viewed by the user. + +3) No Modified Version of the Font Software may use the Reserved Font +Name(s) unless explicit written permission is granted by the corresponding +Copyright Holder. This restriction only applies to the primary font name as +presented to the users. + +4) The name(s) of the Copyright Holder(s) or the Author(s) of the Font +Software shall not be used to promote, endorse or advertise any +Modified Version, except to acknowledge the contribution(s) of the +Copyright Holder(s) and the Author(s) or with their explicit written +permission. + +5) The Font Software, modified or unmodified, in part or in whole, +must be distributed entirely under this license, and must not be +distributed under any other license. The requirement for fonts to +remain under this license does not apply to any document created +using the Font Software. +TERMINATION + +This license becomes null and void if any of the above conditions are +not met. +DISCLAIMER + +THE FONT SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO ANY WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF COPYRIGHT, PATENT, TRADEMARK, OR OTHER RIGHT. IN NO EVENT SHALL THE +COPYRIGHT HOLDER BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +INCLUDING ANY GENERAL, SPECIAL, INDIRECT, INCIDENTAL, OR CONSEQUENTIAL +DAMAGES, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF THE USE OR INABILITY TO USE THE FONT SOFTWARE OR FROM +OTHER DEALINGS IN THE FONT SOFTWARE. diff --git a/licenses-binary/LICENSE.graphlib b/licenses-binary/LICENSE.graphlib new file mode 100644 index 0000000000000..e3c8f95557dbb --- /dev/null +++ b/licenses-binary/LICENSE.graphlib @@ -0,0 +1,19 @@ +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/licenses-binary/LICENSE.grizzled-slf4j b/licenses-binary/LICENSE.grizzled-slf4j new file mode 100644 index 0000000000000..6601c0131eeac --- /dev/null +++ b/licenses-binary/LICENSE.grizzled-slf4j @@ -0,0 +1,11 @@ +Copyright © 2010-2012 Brian M. Clapper. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + +Neither the name “clapper.org” nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.imagesloaded b/licenses-binary/LICENSE.imagesloaded new file mode 100644 index 0000000000000..3e30b1aeff26a --- /dev/null +++ b/licenses-binary/LICENSE.imagesloaded @@ -0,0 +1,7 @@ +Copyright © 2018 David DeSandro + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/licenses-binary/LICENSE.janino b/licenses-binary/LICENSE.janino new file mode 100644 index 0000000000000..ef871e2426218 --- /dev/null +++ b/licenses-binary/LICENSE.janino @@ -0,0 +1,31 @@ +Janino - An embedded Java[TM] compiler + +Copyright (c) 2001-2016, Arno Unkrig +Copyright (c) 2015-2016 TIBCO Software Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials + provided with the distribution. + 3. Neither the name of JANINO nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER +IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR +OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN +IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.jline b/licenses-binary/LICENSE.jline new file mode 100644 index 0000000000000..e34763968c262 --- /dev/null +++ b/licenses-binary/LICENSE.jline @@ -0,0 +1,7 @@ +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.jquery b/licenses-binary/LICENSE.jquery new file mode 100644 index 0000000000000..e3dbacb999cef --- /dev/null +++ b/licenses-binary/LICENSE.jquery @@ -0,0 +1,20 @@ +Copyright JS Foundation and other contributors, https://js.foundation/ + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses-binary/LICENSE.jsch b/licenses-binary/LICENSE.jsch new file mode 100644 index 0000000000000..edd491dfbfbf5 --- /dev/null +++ b/licenses-binary/LICENSE.jsch @@ -0,0 +1,30 @@ +JSch 0.0.* was released under the GNU LGPL license. Later, we have switched +over to a BSD-style license. + +------------------------------------------------------------------------------ +Copyright (c) 2002-2015 Atsuhiko Yamanaka, JCraft,Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.jsr166y b/licenses-binary/LICENSE.jsr166y new file mode 100644 index 0000000000000..b1c292b54cb2b --- /dev/null +++ b/licenses-binary/LICENSE.jsr166y @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuity of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/licenses-binary/LICENSE.jtransforms b/licenses-binary/LICENSE.jtransforms new file mode 100644 index 0000000000000..89169ecf6d12a --- /dev/null +++ b/licenses-binary/LICENSE.jtransforms @@ -0,0 +1,480 @@ + MOZILLA PUBLIC LICENSE + Version 1.1 + + --------------- + +1. Definitions. + + 1.0.1. "Commercial Use" means distribution or otherwise making the + Covered Code available to a third party. + + 1.1. "Contributor" means each entity that creates or contributes to + the creation of Modifications. + + 1.2. "Contributor Version" means the combination of the Original + Code, prior Modifications used by a Contributor, and the Modifications + made by that particular Contributor. + + 1.3. "Covered Code" means the Original Code or Modifications or the + combination of the Original Code and Modifications, in each case + including portions thereof. + + 1.4. "Electronic Distribution Mechanism" means a mechanism generally + accepted in the software development community for the electronic + transfer of data. + + 1.5. "Executable" means Covered Code in any form other than Source + Code. + + 1.6. "Initial Developer" means the individual or entity identified + as the Initial Developer in the Source Code notice required by Exhibit + A. + + 1.7. "Larger Work" means a work which combines Covered Code or + portions thereof with code not governed by the terms of this License. + + 1.8. "License" means this document. + + 1.8.1. "Licensable" means having the right to grant, to the maximum + extent possible, whether at the time of the initial grant or + subsequently acquired, any and all of the rights conveyed herein. + + 1.9. "Modifications" means any addition to or deletion from the + substance or structure of either the Original Code or any previous + Modifications. When Covered Code is released as a series of files, a + Modification is: + A. Any addition to or deletion from the contents of a file + containing Original Code or previous Modifications. + + B. Any new file that contains any part of the Original Code or + previous Modifications. + + 1.10. "Original Code" means Source Code of computer software code + which is described in the Source Code notice required by Exhibit A as + Original Code, and which, at the time of its release under this + License is not already Covered Code governed by this License. + + 1.10.1. "Patent Claims" means any patent claim(s), now owned or + hereafter acquired, including without limitation, method, process, + and apparatus claims, in any patent Licensable by grantor. + + 1.11. "Source Code" means the preferred form of the Covered Code for + making modifications to it, including all modules it contains, plus + any associated interface definition files, scripts used to control + compilation and installation of an Executable, or source code + differential comparisons against either the Original Code or another + well known, available Covered Code of the Contributor's choice. The + Source Code can be in a compressed or archival form, provided the + appropriate decompression or de-archiving software is widely available + for no charge. + + 1.12. "You" (or "Your") means an individual or a legal entity + exercising rights under, and complying with all of the terms of, this + License or a future version of this License issued under Section 6.1. + For legal entities, "You" includes any entity which controls, is + controlled by, or is under common control with You. For purposes of + this definition, "control" means (a) the power, direct or indirect, + to cause the direction or management of such entity, whether by + contract or otherwise, or (b) ownership of more than fifty percent + (50%) of the outstanding shares or beneficial ownership of such + entity. + +2. Source Code License. + + 2.1. The Initial Developer Grant. + The Initial Developer hereby grants You a world-wide, royalty-free, + non-exclusive license, subject to third party intellectual property + claims: + (a) under intellectual property rights (other than patent or + trademark) Licensable by Initial Developer to use, reproduce, + modify, display, perform, sublicense and distribute the Original + Code (or portions thereof) with or without Modifications, and/or + as part of a Larger Work; and + + (b) under Patents Claims infringed by the making, using or + selling of Original Code, to make, have made, use, practice, + sell, and offer for sale, and/or otherwise dispose of the + Original Code (or portions thereof). + + (c) the licenses granted in this Section 2.1(a) and (b) are + effective on the date Initial Developer first distributes + Original Code under the terms of this License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is + granted: 1) for code that You delete from the Original Code; 2) + separate from the Original Code; or 3) for infringements caused + by: i) the modification of the Original Code or ii) the + combination of the Original Code with other software or devices. + + 2.2. Contributor Grant. + Subject to third party intellectual property claims, each Contributor + hereby grants You a world-wide, royalty-free, non-exclusive license + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Contributor, to use, reproduce, modify, + display, perform, sublicense and distribute the Modifications + created by such Contributor (or portions thereof) either on an + unmodified basis, with other Modifications, as Covered Code + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or + selling of Modifications made by that Contributor either alone + and/or in combination with its Contributor Version (or portions + of such combination), to make, use, sell, offer for sale, have + made, and/or otherwise dispose of: 1) Modifications made by that + Contributor (or portions thereof); and 2) the combination of + Modifications made by that Contributor with its Contributor + Version (or portions of such combination). + + (c) the licenses granted in Sections 2.2(a) and 2.2(b) are + effective on the date Contributor first makes Commercial Use of + the Covered Code. + + (d) Notwithstanding Section 2.2(b) above, no patent license is + granted: 1) for any code that Contributor has deleted from the + Contributor Version; 2) separate from the Contributor Version; + 3) for infringements caused by: i) third party modifications of + Contributor Version or ii) the combination of Modifications made + by that Contributor with other software (except as part of the + Contributor Version) or other devices; or 4) under Patent Claims + infringed by Covered Code in the absence of Modifications made by + that Contributor. + +3. Distribution Obligations. + + 3.1. Application of License. + The Modifications which You create or to which You contribute are + governed by the terms of this License, including without limitation + Section 2.2. The Source Code version of Covered Code may be + distributed only under the terms of this License or a future version + of this License released under Section 6.1, and You must include a + copy of this License with every copy of the Source Code You + distribute. You may not offer or impose any terms on any Source Code + version that alters or restricts the applicable version of this + License or the recipients' rights hereunder. However, You may include + an additional document offering the additional rights described in + Section 3.5. + + 3.2. Availability of Source Code. + Any Modification which You create or to which You contribute must be + made available in Source Code form under the terms of this License + either on the same media as an Executable version or via an accepted + Electronic Distribution Mechanism to anyone to whom you made an + Executable version available; and if made available via Electronic + Distribution Mechanism, must remain available for at least twelve (12) + months after the date it initially became available, or at least six + (6) months after a subsequent version of that particular Modification + has been made available to such recipients. You are responsible for + ensuring that the Source Code version remains available even if the + Electronic Distribution Mechanism is maintained by a third party. + + 3.3. Description of Modifications. + You must cause all Covered Code to which You contribute to contain a + file documenting the changes You made to create that Covered Code and + the date of any change. You must include a prominent statement that + the Modification is derived, directly or indirectly, from Original + Code provided by the Initial Developer and including the name of the + Initial Developer in (a) the Source Code, and (b) in any notice in an + Executable version or related documentation in which You describe the + origin or ownership of the Covered Code. + + 3.4. Intellectual Property Matters + (a) Third Party Claims. + If Contributor has knowledge that a license under a third party's + intellectual property rights is required to exercise the rights + granted by such Contributor under Sections 2.1 or 2.2, + Contributor must include a text file with the Source Code + distribution titled "LEGAL" which describes the claim and the + party making the claim in sufficient detail that a recipient will + know whom to contact. If Contributor obtains such knowledge after + the Modification is made available as described in Section 3.2, + Contributor shall promptly modify the LEGAL file in all copies + Contributor makes available thereafter and shall take other steps + (such as notifying appropriate mailing lists or newsgroups) + reasonably calculated to inform those who received the Covered + Code that new knowledge has been obtained. + + (b) Contributor APIs. + If Contributor's Modifications include an application programming + interface and Contributor has knowledge of patent licenses which + are reasonably necessary to implement that API, Contributor must + also include this information in the LEGAL file. + + (c) Representations. + Contributor represents that, except as disclosed pursuant to + Section 3.4(a) above, Contributor believes that Contributor's + Modifications are Contributor's original creation(s) and/or + Contributor has sufficient rights to grant the rights conveyed by + this License. + + 3.5. Required Notices. + You must duplicate the notice in Exhibit A in each file of the Source + Code. If it is not possible to put such notice in a particular Source + Code file due to its structure, then You must include such notice in a + location (such as a relevant directory) where a user would be likely + to look for such a notice. If You created one or more Modification(s) + You may add your name as a Contributor to the notice described in + Exhibit A. You must also duplicate this License in any documentation + for the Source Code where You describe recipients' rights or ownership + rights relating to Covered Code. You may choose to offer, and to + charge a fee for, warranty, support, indemnity or liability + obligations to one or more recipients of Covered Code. However, You + may do so only on Your own behalf, and not on behalf of the Initial + Developer or any Contributor. You must make it absolutely clear than + any such warranty, support, indemnity or liability obligation is + offered by You alone, and You hereby agree to indemnify the Initial + Developer and every Contributor for any liability incurred by the + Initial Developer or such Contributor as a result of warranty, + support, indemnity or liability terms You offer. + + 3.6. Distribution of Executable Versions. + You may distribute Covered Code in Executable form only if the + requirements of Section 3.1-3.5 have been met for that Covered Code, + and if You include a notice stating that the Source Code version of + the Covered Code is available under the terms of this License, + including a description of how and where You have fulfilled the + obligations of Section 3.2. The notice must be conspicuously included + in any notice in an Executable version, related documentation or + collateral in which You describe recipients' rights relating to the + Covered Code. You may distribute the Executable version of Covered + Code or ownership rights under a license of Your choice, which may + contain terms different from this License, provided that You are in + compliance with the terms of this License and that the license for the + Executable version does not attempt to limit or alter the recipient's + rights in the Source Code version from the rights set forth in this + License. If You distribute the Executable version under a different + license You must make it absolutely clear that any terms which differ + from this License are offered by You alone, not by the Initial + Developer or any Contributor. You hereby agree to indemnify the + Initial Developer and every Contributor for any liability incurred by + the Initial Developer or such Contributor as a result of any such + terms You offer. + + 3.7. Larger Works. + You may create a Larger Work by combining Covered Code with other code + not governed by the terms of this License and distribute the Larger + Work as a single product. In such a case, You must make sure the + requirements of this License are fulfilled for the Covered Code. + +4. Inability to Comply Due to Statute or Regulation. + + If it is impossible for You to comply with any of the terms of this + License with respect to some or all of the Covered Code due to + statute, judicial order, or regulation then You must: (a) comply with + the terms of this License to the maximum extent possible; and (b) + describe the limitations and the code they affect. Such description + must be included in the LEGAL file described in Section 3.4 and must + be included with all distributions of the Source Code. Except to the + extent prohibited by statute or regulation, such description must be + sufficiently detailed for a recipient of ordinary skill to be able to + understand it. + +5. Application of this License. + + This License applies to code to which the Initial Developer has + attached the notice in Exhibit A and to related Covered Code. + +6. Versions of the License. + + 6.1. New Versions. + Netscape Communications Corporation ("Netscape") may publish revised + and/or new versions of the License from time to time. Each version + will be given a distinguishing version number. + + 6.2. Effect of New Versions. + Once Covered Code has been published under a particular version of the + License, You may always continue to use it under the terms of that + version. You may also choose to use such Covered Code under the terms + of any subsequent version of the License published by Netscape. No one + other than Netscape has the right to modify the terms applicable to + Covered Code created under this License. + + 6.3. Derivative Works. + If You create or use a modified version of this License (which you may + only do in order to apply it to code which is not already Covered Code + governed by this License), You must (a) rename Your license so that + the phrases "Mozilla", "MOZILLAPL", "MOZPL", "Netscape", + "MPL", "NPL" or any confusingly similar phrase do not appear in your + license (except to note that your license differs from this License) + and (b) otherwise make it clear that Your version of the license + contains terms which differ from the Mozilla Public License and + Netscape Public License. (Filling in the name of the Initial + Developer, Original Code or Contributor in the notice described in + Exhibit A shall not of themselves be deemed to be modifications of + this License.) + +7. DISCLAIMER OF WARRANTY. + + COVERED CODE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, + WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, + WITHOUT LIMITATION, WARRANTIES THAT THE COVERED CODE IS FREE OF + DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. + THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED CODE + IS WITH YOU. SHOULD ANY COVERED CODE PROVE DEFECTIVE IN ANY RESPECT, + YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE + COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER + OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF + ANY COVERED CODE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +8. TERMINATION. + + 8.1. This License and the rights granted hereunder will terminate + automatically if You fail to comply with terms herein and fail to cure + such breach within 30 days of becoming aware of the breach. All + sublicenses to the Covered Code which are properly granted shall + survive any termination of this License. Provisions which, by their + nature, must remain in effect beyond the termination of this License + shall survive. + + 8.2. If You initiate litigation by asserting a patent infringement + claim (excluding declatory judgment actions) against Initial Developer + or a Contributor (the Initial Developer or Contributor against whom + You file such action is referred to as "Participant") alleging that: + + (a) such Participant's Contributor Version directly or indirectly + infringes any patent, then any and all rights granted by such + Participant to You under Sections 2.1 and/or 2.2 of this License + shall, upon 60 days notice from Participant terminate prospectively, + unless if within 60 days after receipt of notice You either: (i) + agree in writing to pay Participant a mutually agreeable reasonable + royalty for Your past and future use of Modifications made by such + Participant, or (ii) withdraw Your litigation claim with respect to + the Contributor Version against such Participant. If within 60 days + of notice, a reasonable royalty and payment arrangement are not + mutually agreed upon in writing by the parties or the litigation claim + is not withdrawn, the rights granted by Participant to You under + Sections 2.1 and/or 2.2 automatically terminate at the expiration of + the 60 day notice period specified above. + + (b) any software, hardware, or device, other than such Participant's + Contributor Version, directly or indirectly infringes any patent, then + any rights granted to You by such Participant under Sections 2.1(b) + and 2.2(b) are revoked effective as of the date You first made, used, + sold, distributed, or had made, Modifications made by that + Participant. + + 8.3. If You assert a patent infringement claim against Participant + alleging that such Participant's Contributor Version directly or + indirectly infringes any patent where such claim is resolved (such as + by license or settlement) prior to the initiation of patent + infringement litigation, then the reasonable value of the licenses + granted by such Participant under Sections 2.1 or 2.2 shall be taken + into account in determining the amount or value of any payment or + license. + + 8.4. In the event of termination under Sections 8.1 or 8.2 above, + all end user license agreements (excluding distributors and resellers) + which have been validly granted by You or any distributor hereunder + prior to termination shall survive termination. + +9. LIMITATION OF LIABILITY. + + UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT + (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL + DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED CODE, + OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR + ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY + CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, + WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER + COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN + INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF + LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY + RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW + PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE + EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO + THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. + +10. U.S. GOVERNMENT END USERS. + + The Covered Code is a "commercial item," as that term is defined in + 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer + software" and "commercial computer software documentation," as such + terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 + C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), + all U.S. Government End Users acquire Covered Code with only those + rights set forth herein. + +11. MISCELLANEOUS. + + This License represents the complete agreement concerning subject + matter hereof. If any provision of this License is held to be + unenforceable, such provision shall be reformed only to the extent + necessary to make it enforceable. This License shall be governed by + California law provisions (except to the extent applicable law, if + any, provides otherwise), excluding its conflict-of-law provisions. + With respect to disputes in which at least one party is a citizen of, + or an entity chartered or registered to do business in the United + States of America, any litigation relating to this License shall be + subject to the jurisdiction of the Federal Courts of the Northern + District of California, with venue lying in Santa Clara County, + California, with the losing party responsible for costs, including + without limitation, court costs and reasonable attorneys' fees and + expenses. The application of the United Nations Convention on + Contracts for the International Sale of Goods is expressly excluded. + Any law or regulation which provides that the language of a contract + shall be construed against the drafter shall not apply to this + License. + +12. RESPONSIBILITY FOR CLAIMS. + + As between Initial Developer and the Contributors, each party is + responsible for claims and damages arising, directly or indirectly, + out of its utilization of rights under this License and You agree to + work with Initial Developer and Contributors to distribute such + responsibility on an equitable basis. Nothing herein is intended or + shall be deemed to constitute any admission of liability. + +13. MULTIPLE-LICENSED CODE. + + Initial Developer may designate portions of the Covered Code as + "Multiple-Licensed". "Multiple-Licensed" means that the Initial + Developer permits you to utilize portions of the Covered Code under + Your choice of the MPL or the alternative licenses, if any, specified + by the Initial Developer in the file described in Exhibit A. + +EXHIBIT A -Mozilla Public License. + + ``The contents of this file are subject to the Mozilla Public License + Version 1.1 (the "License"); you may not use this file except in + compliance with the License. You may obtain a copy of the License at + https://www.mozilla.org/MPL/ + + Software distributed under the License is distributed on an "AS IS" + basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See the + License for the specific language governing rights and limitations + under the License. + + The Original Code is ______________________________________. + + The Initial Developer of the Original Code is ________________________. + Portions created by ______________________ are Copyright (C) ______ + _______________________. All Rights Reserved. + + Contributor(s): ______________________________________. + + Alternatively, the contents of this file may be used under the terms + of the _____ license (the "[___] License"), in which case the + provisions of [______] License are applicable instead of those + above. If you wish to allow use of your version of this file only + under the terms of the [____] License and not to allow others to use + your version of this file under the MPL, indicate your decision by + deleting the provisions above and replace them with the notice and + other provisions required by the [___] License. If you do not delete + the provisions above, a recipient may use your version of this file + under either the MPL or the [___] License." + + [NOTE: The text of this Exhibit A may differ slightly from the text of + the notices in the Source Code files of the Original Code. You should + use the text of this Exhibit A rather than the text found in the + Original Code Source Code for Your Modifications.] + +The contents of this file are subject to the Mozilla Public License Version 1.1 (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.mozilla.org/MPL/ + +Software distributed under the License is distributed on an "AS IS" basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See the License for the specific language governing rights and limitations under the License. + +The Original Code is JTransforms. + +The Initial Developer of the Original Code is +Piotr Wendykier, Emory University. +Portions created by the Initial Developer are Copyright (C) 2007-2009 +the Initial Developer. All Rights Reserved. diff --git a/licenses-binary/LICENSE.jython b/licenses-binary/LICENSE.jython new file mode 100644 index 0000000000000..1c2f7ffeb5bc4 --- /dev/null +++ b/licenses-binary/LICENSE.jython @@ -0,0 +1,17 @@ +PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 + +1. This LICENSE AGREEMENT is between the Python Software Foundation ("PSF"), and the Individual or Organization ("Licensee") accessing and otherwise using this software ("Jython") in source or binary form and its associated documentation. + +2. Subject to the terms and conditions of this License Agreement, PSF hereby grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, analyze, test, perform and/or display publicly, prepare derivative works, distribute, and otherwise use Jython alone or in any derivative version, provided, however, that PSF's License Agreement and PSF's notice of copyright, i.e., "Copyright (c) 2007 Python Software Foundation; All Rights Reserved" are retained in Jython alone or in any derivative version prepared by Licensee. + +3. In the event Licensee prepares a derivative work that is based on or incorporates Jython or any part thereof, and wants to make the derivative work available to others as provided herein, then Licensee hereby agrees to include in any such work a brief summary of the changes made to Jython. + +4. PSF is making Jython available to Licensee on an "AS IS" basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF JYTHON WILL NOT INFRINGE ANY THIRD PARTY RIGHTS. + +5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF JYTHON FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING JYTHON, OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. + +6. This License Agreement will automatically terminate upon a material breach of its terms and conditions. + +7. Nothing in this License Agreement shall be deemed to create any relationship of agency, partnership, or joint venture between PSF and Licensee. This License Agreement does not grant permission to use PSF trademarks or trade name in a trademark sense to endorse or promote products or services of Licensee, or any third party. + +8. By copying, installing or otherwise using Jython, Licensee agrees to be bound by the terms and conditions of this License Agreement. diff --git a/licenses-binary/LICENSE.jzlib b/licenses-binary/LICENSE.jzlib new file mode 100644 index 0000000000000..29ad562af0a5e --- /dev/null +++ b/licenses-binary/LICENSE.jzlib @@ -0,0 +1,26 @@ +Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/licenses-binary/LICENSE.kryo b/licenses-binary/LICENSE.kryo new file mode 100644 index 0000000000000..e1cd88478edf7 --- /dev/null +++ b/licenses-binary/LICENSE.kryo @@ -0,0 +1,10 @@ +Copyright (c) 2008, Nathan Sweet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.lodash b/licenses-binary/LICENSE.lodash new file mode 100644 index 0000000000000..e3a30b4eafbab --- /dev/null +++ b/licenses-binary/LICENSE.lodash @@ -0,0 +1,10 @@ +The MIT License + +Copyright JS Foundation and other contributors + +Based on Underscore.js, copyright Jeremy Ashkenas, +DocumentCloud and Investigative Reporters & Editors + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/lodash/lodash \ No newline at end of file diff --git a/licenses-binary/LICENSE.machinist b/licenses-binary/LICENSE.machinist new file mode 100644 index 0000000000000..c4a7a1dfd55ef --- /dev/null +++ b/licenses-binary/LICENSE.machinist @@ -0,0 +1,19 @@ +Copyright (c) 2011-2014 Erik Osheim, Tom Switzer + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/licenses-binary/LICENSE.minlog b/licenses-binary/LICENSE.minlog new file mode 100644 index 0000000000000..e1cd88478edf7 --- /dev/null +++ b/licenses-binary/LICENSE.minlog @@ -0,0 +1,10 @@ +Copyright (c) 2008, Nathan Sweet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. + * Neither the name of Esoteric Software nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.moment b/licenses-binary/LICENSE.moment new file mode 100644 index 0000000000000..8618b7333d6f5 --- /dev/null +++ b/licenses-binary/LICENSE.moment @@ -0,0 +1,22 @@ +Copyright (c) JS Foundation and other contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses-binary/LICENSE.moment-duration-format b/licenses-binary/LICENSE.moment-duration-format new file mode 100644 index 0000000000000..06ec6fbe07368 --- /dev/null +++ b/licenses-binary/LICENSE.moment-duration-format @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2017 vin-car + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/licenses-binary/LICENSE.paranamer b/licenses-binary/LICENSE.paranamer new file mode 100644 index 0000000000000..522176fbca03e --- /dev/null +++ b/licenses-binary/LICENSE.paranamer @@ -0,0 +1,28 @@ +[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ] + + Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.protobuf b/licenses-binary/LICENSE.protobuf new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/licenses-binary/LICENSE.protobuf @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. diff --git a/licenses-binary/LICENSE.qtip2 b/licenses-binary/LICENSE.qtip2 new file mode 100644 index 0000000000000..0cfb790dc91d3 --- /dev/null +++ b/licenses-binary/LICENSE.qtip2 @@ -0,0 +1,22 @@ +Copyright (c) 2012 Craig Michael Thompson + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses-binary/LICENSE.scala b/licenses-binary/LICENSE.scala new file mode 100644 index 0000000000000..6d8bdabbbe0fb --- /dev/null +++ b/licenses-binary/LICENSE.scala @@ -0,0 +1,11 @@ +Copyright (c) 2002- EPFL +Copyright (c) 2011- Lightbend, Inc. + +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. +Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. +Neither the name of the EPFL nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses-binary/LICENSE.scopt b/licenses-binary/LICENSE.scopt new file mode 100644 index 0000000000000..6d6a875878f4b --- /dev/null +++ b/licenses-binary/LICENSE.scopt @@ -0,0 +1,21 @@ +This project is licensed under the [MIT license](https://en.wikipedia.org/wiki/MIT_License). + +Copyright (c) scopt contributors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE +OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses-binary/LICENSE.slf4j b/licenses-binary/LICENSE.slf4j new file mode 100644 index 0000000000000..93119e70ed5e7 --- /dev/null +++ b/licenses-binary/LICENSE.slf4j @@ -0,0 +1,21 @@ +Copyright (c) 2004-2017 QOS.ch + All rights reserved. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses-binary/LICENSE.slf4j-api b/licenses-binary/LICENSE.slf4j-api new file mode 100644 index 0000000000000..93119e70ed5e7 --- /dev/null +++ b/licenses-binary/LICENSE.slf4j-api @@ -0,0 +1,21 @@ +Copyright (c) 2004-2017 QOS.ch + All rights reserved. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses-binary/LICENSE.spire b/licenses-binary/LICENSE.spire new file mode 100644 index 0000000000000..645cba601fd67 --- /dev/null +++ b/licenses-binary/LICENSE.spire @@ -0,0 +1,19 @@ +Copyright (c) 2011-2012 Erik Osheim, Tom Switzer + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/licenses-binary/LICENSE.split b/licenses-binary/LICENSE.split new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/licenses-binary/LICENSE.split @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. diff --git a/licenses-binary/LICENSE.webbit b/licenses-binary/LICENSE.webbit new file mode 100644 index 0000000000000..ec5f348998bd4 --- /dev/null +++ b/licenses-binary/LICENSE.webbit @@ -0,0 +1,38 @@ +(BSD License: http://www.opensource.org/licenses/bsd-license) + +Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the +following conditions are met: + +* Redistributions of source code must retain the above + copyright notice, this list of conditions and the + following disclaimer. + +* Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the + following disclaimer in the documentation and/or other + materials provided with the distribution. + +* Neither the name of the Webbit nor the names of + its contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND +CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE +GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT +OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + diff --git a/licenses-binary/LICENSE.xmlenc b/licenses-binary/LICENSE.xmlenc new file mode 100644 index 0000000000000..3a70c9bfcdadd --- /dev/null +++ b/licenses-binary/LICENSE.xmlenc @@ -0,0 +1,27 @@ +Copyright 2003-2005, Ernst de Haan +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/pom.xml b/pom.xml index d0e6a77100a48..ab37795ce0765 100644 --- a/pom.xml +++ b/pom.xml @@ -1252,6 +1252,7 @@ under the License. **/packaged_licenses/LICENSE.*.txt **/licenses/LICENSE* + **/licenses-binary/LICENSE* flink-runtime-web/web-dashboard/package.json diff --git a/tools/releasing/collect_license_files.sh b/tools/releasing/collect_license_files.sh new file mode 100755 index 0000000000000..38b11b5c681e6 --- /dev/null +++ b/tools/releasing/collect_license_files.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +# This script extracts from all jars in the specified directory the NOTICE files and the +# licenses folders. It then concatenates all NOTICE files and collects the contents of all +# licenses folders in the specified output directory. +# +# This tool can be used to generate a rough skeleton for the binary NOTICE file. Be aware, +# that it does not deduplicate contents. + +set -Eeuo pipefail + +SRC=${1:-.} +DST=${2:-licenses-output} +PWD=$(pwd) +TMP="${DST}/tmp" + +USAGE="collect_license_files " + +if [ "${SRC}" = "-h" ]; then + echo "${USAGE}" + exit 0 +fi + +for i in $(find -L "${SRC}" -name "*.jar") +do + DIR="${TMP}/$(basename -- "$i" .jar)" + mkdir -p "${DIR}" + JAR="${PWD}/${i}" + (cd "${DIR}" && jar xf ${JAR} META-INF/NOTICE META-INF/licenses) +done + +NOTICE="${DST}/NOTICE" +[ -f "${NOTICE}" ] && rm "${NOTICE}" +find "${TMP}" -name "NOTICE" | xargs cat >> "${NOTICE}" + +LICENSES="${DST}/licenses" +[ -f "${LICENSES}" ] && rm -r "" +find "${TMP}" -name "licenses" -type d -exec cp -r -- "{}" "${DST}" \; + +rm -r "${TMP}" From e74c5d562a6b8829adf6474f036b503aab7ae46a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Nov 2018 16:10:18 +0100 Subject: [PATCH 167/359] [FLINK-10987] Update source LICENSE & NOTICE files The source LICENSE and NOTICE files now follow the same pattern that the LICENSE file only contains the ASL 2.0 license and the dependencies are now listed in the NOTICE file. --- LICENSE | 121 -------------------- NOTICE | 56 +++++++-- licenses/LICENSE.angular | 22 ++++ licenses/LICENSE.angular-drag-and-drop-list | 22 ++++ licenses/LICENSE.angular-moment | 21 ++++ licenses/LICENSE.angular-ui-router | 21 ++++ licenses/LICENSE.bootstrap | 21 ++++ licenses/LICENSE.d3 | 27 +++++ licenses/LICENSE.dagre | 19 +++ licenses/LICENSE.dagre-d3 | 19 +++ licenses/LICENSE.ev-emitter | 7 ++ licenses/LICENSE.font-awesome | 97 ++++++++++++++++ licenses/LICENSE.graphlib | 19 +++ licenses/LICENSE.imagesloaded | 7 ++ licenses/LICENSE.jquery | 20 ++++ licenses/LICENSE.lodash | 10 ++ licenses/LICENSE.moment | 22 ++++ licenses/LICENSE.moment-duration-format | 21 ++++ licenses/LICENSE.qtip2 | 22 ++++ licenses/LICENSE.split | 32 ++++++ 20 files changed, 474 insertions(+), 132 deletions(-) create mode 100644 licenses/LICENSE.angular create mode 100644 licenses/LICENSE.angular-drag-and-drop-list create mode 100644 licenses/LICENSE.angular-moment create mode 100644 licenses/LICENSE.angular-ui-router create mode 100644 licenses/LICENSE.bootstrap create mode 100644 licenses/LICENSE.d3 create mode 100644 licenses/LICENSE.dagre create mode 100644 licenses/LICENSE.dagre-d3 create mode 100644 licenses/LICENSE.ev-emitter create mode 100644 licenses/LICENSE.font-awesome create mode 100644 licenses/LICENSE.graphlib create mode 100644 licenses/LICENSE.imagesloaded create mode 100644 licenses/LICENSE.jquery create mode 100644 licenses/LICENSE.lodash create mode 100644 licenses/LICENSE.moment create mode 100644 licenses/LICENSE.moment-duration-format create mode 100644 licenses/LICENSE.qtip2 create mode 100644 licenses/LICENSE.split diff --git a/LICENSE b/LICENSE index 64850739a2535..261eeb9e9f8b2 100644 --- a/LICENSE +++ b/LICENSE @@ -199,124 +199,3 @@ 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. - - -======================================================================= - -Apache Flink subcomponents: - -The Apache Flink project contains subcomponents in the source code -release with separate copyright notices and license terms. Your use of -the source code for the these subcomponents is subject to the terms and -conditions of their respective licenses. - - ------------------------------------------------------------------------ - The MIT License ------------------------------------------------------------------------ - -The Apache Flink project bundles the following files under the MIT License: - - - Angular.JS v1.4.8 (http://angularjs.org) - Copyright (c) 2010-2015 Google, Inc. - - Angular Moment v0.10.3 (http://www.momentjs.com) - Copyright (C) 2013, 2014, 2015, Uri Shaked - - Angular UI Router v0.2.15 (http://angular-ui.github.io) - Copyright (c) 2013-2015 The AngularUI Team, Karsten Sperling - - angular-drag-and-drop-lists v1.4.0 (https://github.com/marceljuenemann/angular-drag-and-drop-lists) Copyright (c) 2014 Marcel Juenemann , Copyright (c) 2014-2016 Google Inc. - - Bootstrap v3.3.6 (http://getbootstrap.com) - Copyright (c) 2011-2015 Twitter, Inc - - d3 timeline (https://github.com/jiahuang/d3-timeline) - Copyright (c) 2012-2015 Jia Huang - - dagre v0.7.4 (https://github.com/cpettitt/dagre) - Copyright (c) 2012-2014 Chris Pettitt - - dagre-d3 v0.4.17 (https://github.com/cpettitt/dagre-d3) - Copyright (c) 2013 Chris Pettitt - - EvEmitter v1.0.2 (https://github.com/metafizzy/ev-emitter) - Copyright (C) 2016 David DeSandro - - Font Awesome (code) v4.5.0, v4.6.3 (http://fontawesome.io) - Copyright (c) 2014 Dave Gandy - - graphlib v1.0.7 (https://github.com/cpettitt/graphlib) - Copyright (c) 2012-2014 Chris Pettitt - - imagesloaded v4.1.0 (https://github.com/desandro/imagesloaded) - Copyright (C) 2016 David DeSandro - - JQuery v2.2.0 (http://jquery.com/) - Copyright 2014 jQuery Foundation and other contributors - - lodash v3.10.1 (http://dojofoundation.org) - Copyright 2012-2015 The Dojo Foundation - - moment.js v2.10.6 (http://momentjs.com/docs/) - Copyright (c) 2011-2014 Tim Wood, Iskren Chernev, Moment.js contributors - - moment-duration-format v1.3.0 (https://github.com/jsmreese/moment-duration-format) - Copyright (c) 2013 John Madhavan-Reese - - qTip2 v2.2.1 (http://qtip2.com) - Copyright (c) 2012 Craig Michael Thompson - - Split.js v1.0.6 (https://nathancahill.github.io/Split.js/) - Copyright (c) 2015 Nathan Cahill - -All rights reserved. - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in -all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -THE SOFTWARE. - - ------------------------------------------------------------------------ - BSD-style Licenses (BSD-3-clause) ------------------------------------------------------------------------ - -The Apache Flink project bundles the following files under BSD-3-clause licenses: - - D3 v3.5.12 (http://d3js.org/) - Copyright (c) 2010-2015, Michael Bostock - -All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, this list - of conditions and the following disclaimer. - -2. Redistributions in binary form must reproduce the above copyright notice, this - list of conditions and the following disclaimer in the documentation and/or - other materials provided with the distribution. - -3. Neither the name of the copyright holder nor the names of its contributors may - be used to endorse or promote products derived from this software without - specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. -IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, -INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT -NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, -WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. - - ------------------------------------------------------------------------ - The Open Font License ------------------------------------------------------------------------ - -The Apache Flink project bundles the following fonts under the -Open Font License (OFT) - http://scripts.sil.org/OFL - - - Font Awesome (http://fortawesome.github.io/Font-Awesome/) - Created by Dave Gandy - -> fonts in "flink-runtime-web/web-dashboard/web/fonts" - -> fonts in "docs/page/font-awesome/fonts" - ------------------------------------------------------------------------ - The ISC License ------------------------------------------------------------------------ - -The Apache Flink project contains or reuses code that is licensed under the ISC license -from the following projects: - - - simplejmx (http://256stuff.com/sources/simplejmx/) Copyright (c) - Gray Watson - -Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby -granted, provided that this permission notice appear in all copies. - -THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING -ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, -DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, -WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE -USE OR PERFORMANCE OF THIS SOFTWARE. diff --git a/NOTICE b/NOTICE index da67d0e0d9e40..8fd6507657701 100644 --- a/NOTICE +++ b/NOTICE @@ -4,18 +4,52 @@ Copyright 2014-2018 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -======================================================================= +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -Apache Flink contains subcomponents with separate copyright notices and -license terms. Your use of the source code for the these subcomponents -is subject to the terms and conditions of their respective licenses. +- nvd3#1.8.4 -See the LICENSE file for a list of subcomponents and dependencies and -their respective licenses. +This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) +See bundled license files for details. +- angular:1.4.8 +- angular-drag-and-drop-list:1.4.0 +- angular-moment:0.10.3 +- angular-ui-router:0.2.15 +- bootstrap:3.3.6 +- dagre:0.7.5 +- dagre-d3:0.4.17 +- ev-emitter:1.1.1 +- font-awesome:4.5.0 (CSS) +- graphlib:1.0.7 +- imagesloaded:4.1.4 +- jquery:2.2.0 +- lodash:3.10.1 +- moment:2.10.6 +- moment-duration-format:1.3.0 +- qtip2:2.2.1 +- Split.js:1.0.6 -lodash: ---------- -Copyright 2012-2015 The Dojo Foundation -Based on Underscore.js, copyright 2009-2015 Jeremy Ashkenas, -DocumentCloud and Investigative Reporters & Editors +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- d3:3.5.12 + +This project bundles the following dependencies under SIL OFL 1.1 license (https://opensource.org/licenses/OFL-1.1). +See bundled license files for details. + +- font-awesome:4.5.0 (Font) (http://fortawesome.github.io/Font-Awesome/) - Created by Dave Gandy + -> fonts in "flink-runtime-web/web-dashboard/web/fonts" + -> fonts in "docs/page/font-awesome/fonts" + +The Apache Flink project contains or reuses code that is licensed under the ISC license from the following projects. + +- simplejmx (http://256stuff.com/sources/simplejmx/) Copyright (c) - Gray Watson + +Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby +granted, provided that this permission notice appear in all copies. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING +ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, +DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, +WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE +USE OR PERFORMANCE OF THIS SOFTWARE. diff --git a/licenses/LICENSE.angular b/licenses/LICENSE.angular new file mode 100644 index 0000000000000..91f0644936811 --- /dev/null +++ b/licenses/LICENSE.angular @@ -0,0 +1,22 @@ +The MIT License + +Copyright (c) 2010-2018 Google, Inc. http://angularjs.org + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + diff --git a/licenses/LICENSE.angular-drag-and-drop-list b/licenses/LICENSE.angular-drag-and-drop-list new file mode 100644 index 0000000000000..1787fc65befe1 --- /dev/null +++ b/licenses/LICENSE.angular-drag-and-drop-list @@ -0,0 +1,22 @@ +The MIT License (MIT) + +Copyright (c) 2014 Marcel Juenemann +Copyright (c) 2014-2016 Google Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE.angular-moment b/licenses/LICENSE.angular-moment new file mode 100644 index 0000000000000..fe9db79b9113a --- /dev/null +++ b/licenses/LICENSE.angular-moment @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2013-2016 Uri Shaked and contributors + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses/LICENSE.angular-ui-router b/licenses/LICENSE.angular-ui-router new file mode 100644 index 0000000000000..6413b092d70f7 --- /dev/null +++ b/licenses/LICENSE.angular-ui-router @@ -0,0 +1,21 @@ +The MIT License + +Copyright (c) 2013-2015 The AngularUI Team, Karsten Sperling + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses/LICENSE.bootstrap b/licenses/LICENSE.bootstrap new file mode 100644 index 0000000000000..6ca0ceecc5d79 --- /dev/null +++ b/licenses/LICENSE.bootstrap @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) 2011-2018 Twitter, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses/LICENSE.d3 b/licenses/LICENSE.d3 new file mode 100644 index 0000000000000..1d9d875edb469 --- /dev/null +++ b/licenses/LICENSE.d3 @@ -0,0 +1,27 @@ +Copyright 2010-2017 Mike Bostock +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +* Neither the name of the author nor the names of contributors may be used to + endorse or promote products derived from this software without specific prior + written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses/LICENSE.dagre b/licenses/LICENSE.dagre new file mode 100644 index 0000000000000..e3c8f95557dbb --- /dev/null +++ b/licenses/LICENSE.dagre @@ -0,0 +1,19 @@ +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE.dagre-d3 b/licenses/LICENSE.dagre-d3 new file mode 100644 index 0000000000000..1d64ed68ce64d --- /dev/null +++ b/licenses/LICENSE.dagre-d3 @@ -0,0 +1,19 @@ +Copyright (c) 2013 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/licenses/LICENSE.ev-emitter b/licenses/LICENSE.ev-emitter new file mode 100644 index 0000000000000..3e30b1aeff26a --- /dev/null +++ b/licenses/LICENSE.ev-emitter @@ -0,0 +1,7 @@ +Copyright © 2018 David DeSandro + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE.font-awesome b/licenses/LICENSE.font-awesome new file mode 100644 index 0000000000000..b9fb2c6e6ef06 --- /dev/null +++ b/licenses/LICENSE.font-awesome @@ -0,0 +1,97 @@ +The MIT license + +Copyright + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + + + SIL OPEN FONT LICENSE + +Version 1.1 - 26 February 2007 +PREAMBLE + +The goals of the Open Font License (OFL) are to stimulate worldwide +development of collaborative font projects, to support the font creation +efforts of academic and linguistic communities, and to provide a free and +open framework in which fonts may be shared and improved in partnership +with others. + +The OFL allows the licensed fonts to be used, studied, modified and +redistributed freely as long as they are not sold by themselves. The +fonts, including any derivative works, can be bundled, embedded, +redistributed and/or sold with any software provided that any reserved +names are not used by derivative works. The fonts and derivatives, +however, cannot be released under any other type of license. The +requirement for fonts to remain under this license does not apply +to any document created using the fonts or their derivatives. +DEFINITIONS + +"Font Software" refers to the set of files released by the Copyright +Holder(s) under this license and clearly marked as such. This may +include source files, build scripts and documentation. + +"Reserved Font Name" refers to any names specified as such after the +copyright statement(s). + +"Original Version" refers to the collection of Font Software components as +distributed by the Copyright Holder(s). + +"Modified Version" refers to any derivative made by adding to, deleting, +or substituting — in part or in whole — any of the components of the +Original Version, by changing formats or by porting the Font Software to a +new environment. + +"Author" refers to any designer, engineer, programmer, technical +writer or other person who contributed to the Font Software. +PERMISSION & CONDITIONS + +Permission is hereby granted, free of charge, to any person obtaining +a copy of the Font Software, to use, study, copy, merge, embed, modify, +redistribute, and sell modified and unmodified copies of the Font +Software, subject to the following conditions: + +1) Neither the Font Software nor any of its individual components, +in Original or Modified Versions, may be sold by itself. + +2) Original or Modified Versions of the Font Software may be bundled, +redistributed and/or sold with any software, provided that each copy +contains the above copyright notice and this license. These can be +included either as stand-alone text files, human-readable headers or +in the appropriate machine-readable metadata fields within text or +binary files as long as those fields can be easily viewed by the user. + +3) No Modified Version of the Font Software may use the Reserved Font +Name(s) unless explicit written permission is granted by the corresponding +Copyright Holder. This restriction only applies to the primary font name as +presented to the users. + +4) The name(s) of the Copyright Holder(s) or the Author(s) of the Font +Software shall not be used to promote, endorse or advertise any +Modified Version, except to acknowledge the contribution(s) of the +Copyright Holder(s) and the Author(s) or with their explicit written +permission. + +5) The Font Software, modified or unmodified, in part or in whole, +must be distributed entirely under this license, and must not be +distributed under any other license. The requirement for fonts to +remain under this license does not apply to any document created +using the Font Software. +TERMINATION + +This license becomes null and void if any of the above conditions are +not met. +DISCLAIMER + +THE FONT SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO ANY WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF COPYRIGHT, PATENT, TRADEMARK, OR OTHER RIGHT. IN NO EVENT SHALL THE +COPYRIGHT HOLDER BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +INCLUDING ANY GENERAL, SPECIAL, INDIRECT, INCIDENTAL, OR CONSEQUENTIAL +DAMAGES, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF THE USE OR INABILITY TO USE THE FONT SOFTWARE OR FROM +OTHER DEALINGS IN THE FONT SOFTWARE. diff --git a/licenses/LICENSE.graphlib b/licenses/LICENSE.graphlib new file mode 100644 index 0000000000000..e3c8f95557dbb --- /dev/null +++ b/licenses/LICENSE.graphlib @@ -0,0 +1,19 @@ +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE.imagesloaded b/licenses/LICENSE.imagesloaded new file mode 100644 index 0000000000000..3e30b1aeff26a --- /dev/null +++ b/licenses/LICENSE.imagesloaded @@ -0,0 +1,7 @@ +Copyright © 2018 David DeSandro + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the “Software”), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE.jquery b/licenses/LICENSE.jquery new file mode 100644 index 0000000000000..e3dbacb999cef --- /dev/null +++ b/licenses/LICENSE.jquery @@ -0,0 +1,20 @@ +Copyright JS Foundation and other contributors, https://js.foundation/ + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses/LICENSE.lodash b/licenses/LICENSE.lodash new file mode 100644 index 0000000000000..e3a30b4eafbab --- /dev/null +++ b/licenses/LICENSE.lodash @@ -0,0 +1,10 @@ +The MIT License + +Copyright JS Foundation and other contributors + +Based on Underscore.js, copyright Jeremy Ashkenas, +DocumentCloud and Investigative Reporters & Editors + +This software consists of voluntary contributions made by many +individuals. For exact contribution history, see the revision history +available at https://github.com/lodash/lodash \ No newline at end of file diff --git a/licenses/LICENSE.moment b/licenses/LICENSE.moment new file mode 100644 index 0000000000000..8618b7333d6f5 --- /dev/null +++ b/licenses/LICENSE.moment @@ -0,0 +1,22 @@ +Copyright (c) JS Foundation and other contributors + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses/LICENSE.moment-duration-format b/licenses/LICENSE.moment-duration-format new file mode 100644 index 0000000000000..06ec6fbe07368 --- /dev/null +++ b/licenses/LICENSE.moment-duration-format @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2017 vin-car + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. diff --git a/licenses/LICENSE.qtip2 b/licenses/LICENSE.qtip2 new file mode 100644 index 0000000000000..0cfb790dc91d3 --- /dev/null +++ b/licenses/LICENSE.qtip2 @@ -0,0 +1,22 @@ +Copyright (c) 2012 Craig Michael Thompson + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without +restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the +Software is furnished to do so, subject to the following +conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES +OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT +HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR +OTHER DEALINGS IN THE SOFTWARE. diff --git a/licenses/LICENSE.split b/licenses/LICENSE.split new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/licenses/LICENSE.split @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. From d311cc4d55be6a83badda903c4d15727a901f70e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Nov 2018 18:55:54 +0100 Subject: [PATCH 168/359] [hotfix] Remove flink-metrics-ganglia from flink-dist/pom.xml --- flink-dist/pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index fc0bb02d2077a..bd5d6bccc2f6f 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -228,13 +228,6 @@ under the License. provided - - org.apache.flink - flink-metrics-ganglia - ${project.version} - provided - - org.apache.flink flink-metrics-graphite From 64b4ef513a0dbee307dfd212e3bcbcb9bd1eba90 Mon Sep 17 00:00:00 2001 From: zhijiang Date: Tue, 27 Nov 2018 22:27:12 +0800 Subject: [PATCH 169/359] [FLINK-10367][network] Introduce NotificationResult for BufferListener to solve recursive stack overflow (#6829) Before this fix, during LocalBufferPool#recycle, the recycled buffer would be notified to a BufferListener. But this BufferListener may not need floating buffers any more currently, so this buffer is recycled again to the LocalBufferPool, then another BufferListener is selected to be notified of this available buffer. The above process may be repeatedly triggered in recursive way that will cause stack overflow error in extreme case. We ever encountered this error triggered by release all resources during task failover in large scale job, especially it will also result in buffer leak after stack overflow. This fix removes recursive calls and replace them with a loop and adds a `NotificationResult` as a result from BufferListener#notifyBufferAvailable to inform the caller whether the buffer was used or not. --- .../io/network/buffer/BufferListener.java | 39 ++++++++++++++++++- .../io/network/buffer/LocalBufferPool.java | 36 +++++++++-------- .../netty/PartitionRequestClientHandler.java | 32 +++++---------- .../consumer/RemoteInputChannel.java | 27 +++++-------- .../network/buffer/LocalBufferPoolTest.java | 8 +++- .../consumer/RemoteInputChannelTest.java | 6 ++- 6 files changed, 86 insertions(+), 62 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java index 4cc32c0a66147..e6b5416d98662 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferListener.java @@ -24,6 +24,41 @@ */ public interface BufferListener { + /** + * Status of the notification result from the buffer listener. + */ + enum NotificationResult { + BUFFER_NOT_USED(false, false), + BUFFER_USED_NO_NEED_MORE(true, false), + BUFFER_USED_NEED_MORE(true, true); + + private final boolean isBufferUsed; + private final boolean needsMoreBuffers; + + NotificationResult(boolean isBufferUsed, boolean needsMoreBuffers) { + this.isBufferUsed = isBufferUsed; + this.needsMoreBuffers = needsMoreBuffers; + } + + /** + * Whether the notified buffer is accepted to use by the listener. + * + * @return true if the notified buffer is accepted. + */ + boolean isBufferUsed() { + return isBufferUsed; + } + + /** + * Whether the listener still needs more buffers to be notified. + * + * @return true if the listener is still waiting for more buffers. + */ + boolean needsMoreBuffers() { + return needsMoreBuffers; + } + } + /** * Notification callback if a buffer is recycled and becomes available in buffer pool. * @@ -37,9 +72,9 @@ public interface BufferListener { * stack! * * @param buffer buffer that becomes available in buffer pool. - * @return true if the listener wants to be notified next time. + * @return NotificationResult if the listener wants to be notified next time. */ - boolean notifyBufferAvailable(Buffer buffer); + NotificationResult notifyBufferAvailable(Buffer buffer); /** * Notification callback if the buffer provider is destroyed. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index d7bfb603eafa9..dfad5b9cfaba0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.buffer; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.BufferListener.NotificationResult; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -271,30 +272,31 @@ private MemorySegment requestMemorySegment(boolean isBlocking) throws Interrupte @Override public void recycle(MemorySegment segment) { BufferListener listener; - synchronized (availableMemorySegments) { - if (isDestroyed || numberOfRequestedMemorySegments > currentPoolSize) { - returnMemorySegment(segment); - return; - } else { - listener = registeredListeners.poll(); - - if (listener == null) { - availableMemorySegments.add(segment); - availableMemorySegments.notify(); + NotificationResult notificationResult = NotificationResult.BUFFER_NOT_USED; + while (!notificationResult.isBufferUsed()) { + synchronized (availableMemorySegments) { + if (isDestroyed || numberOfRequestedMemorySegments > currentPoolSize) { + returnMemorySegment(segment); return; + } else { + listener = registeredListeners.poll(); + if (listener == null) { + availableMemorySegments.add(segment); + availableMemorySegments.notify(); + return; + } } } + notificationResult = fireBufferAvailableNotification(listener, segment); } + } + private NotificationResult fireBufferAvailableNotification(BufferListener listener, MemorySegment segment) { // We do not know which locks have been acquired before the recycle() or are needed in the // notification and which other threads also access them. // -> call notifyBufferAvailable() outside of the synchronized block to avoid a deadlock (FLINK-9676) - // Note that in case of any exceptions notifyBufferAvailable() should recycle the buffer - // (either directly or later during error handling) and therefore eventually end up in this - // method again. - boolean needMoreBuffers = listener.notifyBufferAvailable(new NetworkBuffer(segment, this)); - - if (needMoreBuffers) { + NotificationResult notificationResult = listener.notifyBufferAvailable(new NetworkBuffer(segment, this)); + if (notificationResult.needsMoreBuffers()) { synchronized (availableMemorySegments) { if (isDestroyed) { // cleanup tasks how they would have been done if we only had one synchronized block @@ -304,6 +306,7 @@ public void recycle(MemorySegment segment) { } } } + return notificationResult; } /** @@ -404,5 +407,4 @@ private void returnExcessMemorySegments() { returnMemorySegment(segment); } } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java index c5ba7a4b7f1ac..34f65c0f22dc1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java @@ -417,32 +417,18 @@ public void notifyBufferDestroyed() { // Called by the recycling thread (not network I/O thread) @Override - public boolean notifyBufferAvailable(Buffer buffer) { - boolean success = false; - - try { - if (availableBuffer.compareAndSet(null, buffer)) { - ctx.channel().eventLoop().execute(this); + public NotificationResult notifyBufferAvailable(Buffer buffer) { + if (availableBuffer.compareAndSet(null, buffer)) { + ctx.channel().eventLoop().execute(this); - success = true; - } - else { - throw new IllegalStateException("Received a buffer notification, " + - " but the previous one has not been handled yet."); - } - } - catch (Throwable t) { - ctx.channel().eventLoop().execute(new AsyncErrorNotificationTask(t)); + return NotificationResult.BUFFER_USED_NO_NEED_MORE; } - finally { - if (!success) { - if (buffer != null) { - buffer.recycleBuffer(); - } - } + else { + ctx.channel().eventLoop().execute(new AsyncErrorNotificationTask( + new IllegalStateException("Received a buffer notification, " + + " but the previous one has not been handled yet."))); + return NotificationResult.BUFFER_NOT_USED; } - - return false; } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 6738abd7f9c05..141494996c792 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -342,18 +342,18 @@ boolean isWaitingForFloatingBuffers() { /** * The Buffer pool notifies this channel of an available floating buffer. If the channel is released or - * currently does not need extra buffers, the buffer should be recycled to the buffer pool. Otherwise, + * currently does not need extra buffers, the buffer should be returned to the buffer pool. Otherwise, * the buffer will be added into the bufferQueue and the unannounced credit is increased * by one. * * @param buffer Buffer that becomes available in buffer pool. - * @return True when this channel is waiting for more floating buffers, otherwise false. + * @return NotificationResult indicates whether this channel accepts the buffer and is waiting for + * more floating buffers. */ @Override - public boolean notifyBufferAvailable(Buffer buffer) { - boolean recycleBuffer = true; + public NotificationResult notifyBufferAvailable(Buffer buffer) { + NotificationResult notificationResult = NotificationResult.BUFFER_NOT_USED; try { - boolean needMoreBuffers = false; synchronized (bufferQueue) { checkState(isWaitingForFloatingBuffers, "This channel should be waiting for floating buffers."); @@ -364,36 +364,29 @@ public boolean notifyBufferAvailable(Buffer buffer) { // -> then isReleased is set correctly // 2) releaseAllResources() did not yet release buffers from bufferQueue // -> we may or may not have set isReleased yet but will always wait for the - // lock on bufferQueue to release buffers + // lock on bufferQueue to release buffers if (isReleased.get() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { isWaitingForFloatingBuffers = false; - recycleBuffer = false; // just in case - buffer.recycleBuffer(); - return false; + return notificationResult; } - recycleBuffer = false; bufferQueue.addFloatingBuffer(buffer); if (bufferQueue.getAvailableBufferSize() == numRequiredBuffers) { isWaitingForFloatingBuffers = false; + notificationResult = NotificationResult.BUFFER_USED_NO_NEED_MORE; } else { - needMoreBuffers = true; + notificationResult = NotificationResult.BUFFER_USED_NEED_MORE; } } if (unannouncedCredit.getAndAdd(1) == 0) { notifyCreditAvailable(); } - - return needMoreBuffers; } catch (Throwable t) { - if (recycleBuffer) { - buffer.recycleBuffer(); - } setError(t); - return false; } + return notificationResult; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index 537d167908f2a..a0e10d7c6872a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -414,10 +414,14 @@ private BufferListener createBufferListener(int notificationTimes) { AtomicInteger times = new AtomicInteger(0); @Override - public boolean notifyBufferAvailable(Buffer buffer) { + public NotificationResult notifyBufferAvailable(Buffer buffer) { int newCount = times.incrementAndGet(); buffer.recycleBuffer(); - return newCount < notificationTimes; + if (newCount < notificationTimes) { + return NotificationResult.BUFFER_USED_NEED_MORE; + } else { + return NotificationResult.BUFFER_USED_NO_NEED_MORE; + } } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index ec80459f0ead4..7747421fc7b5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferListener.NotificationResult; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.netty.PartitionRequestClient; @@ -150,7 +151,10 @@ private void testConcurrentReleaseAndSomething( for (int j = 0; j < 128; j++) { // this is the same buffer over and over again which will be // recycled by the RemoteInputChannel - function.apply(inputChannel, buffer.retainBuffer(), j); + Object obj = function.apply(inputChannel, buffer.retainBuffer(), j); + if (obj instanceof NotificationResult && obj == NotificationResult.BUFFER_NOT_USED) { + buffer.recycleBuffer(); + } } if (inputChannel.isReleased()) { From e9276d20d62ab13ba6002a707e611fe440523792 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 29 Nov 2018 15:38:02 +0100 Subject: [PATCH 170/359] [hotfix] Let CoGroupGroupSortTranslationTest extend TestLogger and fix testGroupSortTuplesDefaultCoGroup --- .../CoGroupGroupSortTranslationTest.scala | 30 ++++++++----------- 1 file changed, 12 insertions(+), 18 deletions(-) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala index a0f93dd5dcb35..bf07aa0437fec 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala @@ -18,22 +18,15 @@ package org.apache.flink.api.scala.operators.translation -import org.apache.flink.api.java.io.DiscardingOutputFormat -import org.apache.flink.optimizer.util.CompilerTestBase -import org.junit.Assert._ -import org.junit.Test -import org.apache.flink.api.common.functions.Partitioner -import org.apache.flink.api.scala._ -import org.apache.flink.runtime.operators.shipping.ShipStrategyType -import org.apache.flink.optimizer.plan.SingleInputPlanNode import org.apache.flink.api.common.operators.Order -import org.apache.flink.api.common.InvalidProgramException -import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint -import org.apache.flink.optimizer.plan.DualInputPlanNode import org.apache.flink.api.common.operators.base.CoGroupOperatorBase -import org.junit.Ignore +import org.apache.flink.api.java.io.DiscardingOutputFormat +import org.apache.flink.api.scala._ +import org.apache.flink.util.{Collector, TestLogger} +import org.junit.Assert._ +import org.junit.{Ignore, Test} -class CoGroupGroupSortTranslationTest { +class CoGroupGroupSortTranslationTest extends TestLogger { @Test def testGroupSortTuples() { @@ -131,11 +124,12 @@ class CoGroupGroupSortTranslationTest { val input2 = env.fromElements( (0L, 0L, 0L) ) input1 - .coGroup(input2) - .where(1).equalTo(2) - .sortFirstGroup(0, Order.DESCENDING) - .sortSecondGroup(1, Order.ASCENDING).sortSecondGroup(0, Order.DESCENDING) - .print() + .coGroup(input2) + .where(1).equalTo(2) + .sortFirstGroup(0, Order.DESCENDING) + .sortSecondGroup(1, Order.ASCENDING).sortSecondGroup(0, Order.DESCENDING) + .apply((a, b, c: Collector[(Long, Long)]) => a.foreach(e => c.collect(e))) + .output(new DiscardingOutputFormat[(Long, Long)]) val p = env.createProgramPlan() From 60aa201590c41b64c2d2c7b7fcaddc3375ca08e3 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 29 Nov 2018 17:00:56 +0100 Subject: [PATCH 171/359] [hotfix] Add release notes 1.7 link to documentation front page --- docs/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/index.md b/docs/index.md index e138bc35457bc..031192413c645 100644 --- a/docs/index.md +++ b/docs/index.md @@ -47,6 +47,7 @@ Before putting your Flink job into production, read the [Production Readiness Ch Release notes cover important changes between Flink versions. Please carefully read these notes if you plan to upgrade your Flink setup to a later version. +* [Release notes for Flink 1.7](release-notes/flink-1.7.html). * [Release notes for Flink 1.6](release-notes/flink-1.6.html). * [Release notes for Flink 1.5](release-notes/flink-1.5.html). From b58aaea098afb791b911bf348c0226608ef0fe23 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 29 Nov 2018 17:06:10 +0100 Subject: [PATCH 172/359] [hotfix] Update docs/_config.yml to set correct 1.7 constants/paths --- docs/_config.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index fac4aabccbb43..ba4328611faa7 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -27,12 +27,12 @@ # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) -version: "1.7-SNAPSHOT" +version: "1.7.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version -version_title: "1.7-SNAPSHOT" +version_title: "1.7" # Branch on Github for this version -github_branch: "master" +github_branch: "release-1.7" # This suffix is appended to the Scala-dependent Maven artifact names scala_version_suffix: "_2.11" @@ -45,10 +45,10 @@ github_url: "https://github.com/apache/flink" download_url: "http://flink.apache.org/downloads.html" # please use a protocol relative URL here -baseurl: //ci.apache.org/projects/flink/flink-docs-master +baseurl: //ci.apache.org/projects/flink/flink-docs-release-1.7 stable_baseurl: //ci.apache.org/projects/flink/flink-docs-stable -javadocs_baseurl: //ci.apache.org/projects/flink/flink-docs-master +javadocs_baseurl: //ci.apache.org/projects/flink/flink-docs-release-1.7 # Flag whether this is a stable version or not. Used for the quickstart page. is_stable: false From 012ff7adb3343a44288f423faeb7c56a003eb467 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 30 Nov 2018 11:17:44 +0100 Subject: [PATCH 173/359] [hotfix] Bump japicmp Flink reference version to 1.6.2 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ab37795ce0765..60dae4131b50d 100644 --- a/pom.xml +++ b/pom.xml @@ -1737,7 +1737,7 @@ under the License. org.apache.flink ${project.artifactId} - 1.6.1 + 1.6.2 ${project.packaging} From 6e56a3bc813eabe114fd50111f3ad569696e4c4e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 30 Nov 2018 11:18:07 +0100 Subject: [PATCH 174/359] [hotfix] Mark release-1.7 documentation as stable --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index ba4328611faa7..93c67d0e88be3 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -51,7 +51,7 @@ stable_baseurl: //ci.apache.org/projects/flink/flink-docs-stable javadocs_baseurl: //ci.apache.org/projects/flink/flink-docs-release-1.7 # Flag whether this is a stable version or not. Used for the quickstart page. -is_stable: false +is_stable: true # Flag to indicate whether an outdated warning should be shown. show_outdated_warning: false From 8c51551cc08a874acda65cdd697ec0ff30a2293c Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 30 Nov 2018 09:10:16 +0100 Subject: [PATCH 175/359] [hotfix][table] Check if expected exception was thrown --- .../WindowAggregateValidationTest.scala | 28 +++++++++++++++---- 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala index e7cf597ca64b0..a0157d7191a80 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala @@ -30,8 +30,11 @@ class WindowAggregateValidationTest extends TableTestBase { streamUtil.addTable[(Int, String, Long)]( "MyTable", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime) - @Test(expected = classOf[TableException]) + @Test def testTumbleWindowNoOffset(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage("TUMBLE window with alignment is not supported yet") + val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " + "FROM MyTable " + @@ -40,8 +43,11 @@ class WindowAggregateValidationTest extends TableTestBase { streamUtil.verifySql(sqlQuery, "n/a") } - @Test(expected = classOf[TableException]) + @Test def testHopWindowNoOffset(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage("HOP window with alignment is not supported yet") + val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " + "FROM MyTable " + @@ -50,8 +56,11 @@ class WindowAggregateValidationTest extends TableTestBase { streamUtil.verifySql(sqlQuery, "n/a") } - @Test(expected = classOf[TableException]) + @Test def testSessionWindowNoOffset(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage("SESSION window with alignment is not supported yet") + val sqlQuery = "SELECT SUM(a) AS sumA, COUNT(b) AS cntB " + "FROM MyTable " + @@ -60,20 +69,27 @@ class WindowAggregateValidationTest extends TableTestBase { streamUtil.verifySql(sqlQuery, "n/a") } - @Test(expected = classOf[TableException]) + @Test def testVariableWindowSize() = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage("Only constant window descriptors with DAY TO SECOND " + + "resolution are supported") + val sql = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime, c * INTERVAL '1' MINUTE)" streamUtil.verifySql(sql, "n/a") } - @Test(expected = classOf[ValidationException]) + @Test def testWindowUdAggInvalidArgs(): Unit = { + expectedException.expect(classOf[ValidationException]) + expectedException.expectMessage("Given parameters of function do not match any signature") + streamUtil.tableEnv.registerFunction("weightedAvg", new WeightedAvgWithMerge) val sqlQuery = "SELECT SUM(a) AS sumA, weightedAvg(a, b) AS wAvg " + "FROM MyTable " + - "GROUP BY TUMBLE(proctime(), INTERVAL '2' HOUR, TIME '10:00:00')" + "GROUP BY TUMBLE(proctime, INTERVAL '2' HOUR, TIME '10:00:00')" streamUtil.verifySql(sqlQuery, "n/a") } From a583b4a71a10322809400a87840e412c1c0c20d6 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 30 Nov 2018 09:11:12 +0100 Subject: [PATCH 176/359] [FLINK-11017][table] Throw exception if constant with YEAR TO MONTH resolution was used for group windows This closes #7200. --- ...DataStreamLogicalWindowAggregateRule.scala | 8 +++++--- .../WindowAggregateValidationTest.scala | 19 ++++++++++++++++--- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala index 62eecbb053217..c8c58adf22682 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala @@ -22,7 +22,7 @@ import java.math.{BigDecimal => JBigDecimal} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex._ -import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.calcite.sql.`type`.{SqlTypeFamily, SqlTypeName} import org.apache.flink.table.api.scala.{Session, Slide, Tumble} import org.apache.flink.table.api.{TableException, ValidationException, Window} import org.apache.flink.table.calcite.FlinkTypeFactory @@ -68,8 +68,10 @@ class DataStreamLogicalWindowAggregateRule def getOperandAsLong(call: RexCall, idx: Int): Long = call.getOperands.get(idx) match { - case v: RexLiteral => v.getValue.asInstanceOf[JBigDecimal].longValue() - case _ => throw new TableException("Only constant window descriptors are supported.") + case v: RexLiteral if v.getTypeName.getFamily == SqlTypeFamily.INTERVAL_DAY_TIME => + v.getValue.asInstanceOf[JBigDecimal].longValue() + case _ => throw new TableException( + "Only constant window intervals with millisecond resolution are supported.") } def getOperandAsTimeIndicator(call: RexCall, idx: Int): ResolvedFieldReference = diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala index a0157d7191a80..5c237ffc5ee95 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/WindowAggregateValidationTest.scala @@ -70,10 +70,10 @@ class WindowAggregateValidationTest extends TableTestBase { } @Test - def testVariableWindowSize() = { + def testVariableWindowSize(): Unit = { expectedException.expect(classOf[TableException]) - expectedException.expectMessage("Only constant window descriptors with DAY TO SECOND " + - "resolution are supported") + expectedException.expectMessage( + "Only constant window intervals with millisecond resolution are supported") val sql = "SELECT COUNT(*) FROM MyTable GROUP BY TUMBLE(proctime, c * INTERVAL '1' MINUTE)" streamUtil.verifySql(sql, "n/a") @@ -93,4 +93,17 @@ class WindowAggregateValidationTest extends TableTestBase { streamUtil.verifySql(sqlQuery, "n/a") } + + @Test + def testWindowWrongWindowParameter(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage( + "Only constant window intervals with millisecond resolution are supported") + + val sqlQuery = + "SELECT COUNT(*) FROM MyTable " + + "GROUP BY TUMBLE(proctime, INTERVAL '2-10' YEAR TO MONTH)" + + streamUtil.verifySql(sqlQuery, "n/a") + } } From 13e0736106be554d145619a2db6ee758670aab5a Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Thu, 29 Nov 2018 15:13:26 +0800 Subject: [PATCH 177/359] [hotfix] [docs] Fix typos in MATCH_RECOGNIZE docs This closes #7194. --- docs/dev/table/streaming/match_recognize.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/dev/table/streaming/match_recognize.md b/docs/dev/table/streaming/match_recognize.md index b12cbe5e0d927..3cd1ed09cc9a2 100644 --- a/docs/dev/table/streaming/match_recognize.md +++ b/docs/dev/table/streaming/match_recognize.md @@ -149,7 +149,7 @@ MATCH_RECOGNIZE ( DEFINE PRICE_DOWN AS (LAST(PRICE_DOWN.price, 1) IS NULL AND PRICE_DOWN.price < START_ROW.price) OR - PRICE_DOWN.price < LAST(PRICE_DOWN.price, 1) + PRICE_DOWN.price < LAST(PRICE_DOWN.price, 1), PRICE_UP AS PRICE_UP.price > LAST(PRICE_DOWN.price, 1) ) MR; @@ -260,8 +260,8 @@ FROM Ticker ONE ROW PER MATCH AFTER MATCH SKIP PAST LAST ROW DEFINE - A AS A.price > 10 - B AS B.price < 15 + A AS A.price > 10, + B AS B.price < 15, C AS B.price > 12 ) {% endhighlight %} @@ -304,8 +304,8 @@ variable of a pattern. Thus, a pattern like `(A B*)` is not allowed. This can be {% highlight sql %} PATTERN (A B* C) DEFINE - A AS condA() - B AS condB() + A AS condA(), + B AS condB(), C AS NOT condB() {% endhighlight %} @@ -331,8 +331,8 @@ FROM Ticker PARTITION BY symbol ORDER BY rowtime MEASURES - FIRST(A.price) AS startPrice - LAST(A.price) AS topPrice + FIRST(A.price) AS startPrice, + LAST(A.price) AS topPrice, B.price AS lastPrice PATTERN (A+ B) ONE ROW PER MATCH @@ -366,13 +366,13 @@ The pattern recognition is partitioned by the `symbol` column. Even though not e Pattern Navigation ------------------ -The `DEFINE` and `MEASURE` clauses allow for navigating within the list of rows that (potentially) match a pattern. +The `DEFINE` and `MEASURES` clauses allow for navigating within the list of rows that (potentially) match a pattern. This section discusses this navigation for declaring conditions or producing output results. ### Pattern Variable Referencing -A _pattern variable reference_ allows a set of rows mapped to a particular pattern variable in the `DEFINE` or `MEASURE` clauses to be referenced. +A _pattern variable reference_ allows a set of rows mapped to a particular pattern variable in the `DEFINE` or `MEASURES` clauses to be referenced. For example, the expression `A.price` describes a set of rows mapped so far to `A` plus the current row if we try to match the current row to `A`. If an expression in the `DEFINE`/`MEASURES` clause requires a single row (e.g. `A.price` or `A.price > 10`), From e82df4f07f0a4e82410db0d30013b8df32e66720 Mon Sep 17 00:00:00 2001 From: Rune Skou Larsen Date: Wed, 15 Aug 2018 12:33:54 +0200 Subject: [PATCH 178/359] [FLINK-10149][mesos] Don't allocate extra mesos port for TM unless configured to do so. --- .../flink/mesos/configuration/MesosOptions.java | 5 +++-- .../clusterframework/LaunchableMesosWorker.java | 10 ++++++---- .../LaunchableMesosWorkerTest.java | 15 +++++++++++++++ 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java b/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java index 426a891e81423..0c4e1f6bcba4b 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java @@ -136,8 +136,9 @@ public class MesosOptions { /** * Config parameter to configure which configuration keys will dynamically get a port assigned through Mesos. */ - public static final ConfigOption PORT_ASSIGNMENTS = key("mesos.resourcemanager.tasks.port-assignments") - .defaultValue("") + public static final ConfigOption PORT_ASSIGNMENTS = + key("mesos.resourcemanager.tasks.port-assignments") + .noDefaultValue() .withDescription(Description.builder() .text("Comma-separated list of configuration keys which represent a configurable port. " + "All port keys will dynamically get a port assigned through Mesos.") diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 84ec2229a2ad4..35bea9918eafe 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -346,10 +346,12 @@ static Set extractPortKeys(Configuration config) { final String portKeys = config.getString(PORT_ASSIGNMENTS); - Arrays.stream(portKeys.split(",")) - .map(String::trim) - .peek(key -> LOG.debug("Adding port key " + key + " to mesos request")) - .forEach(tmPortKeys::add); + if (portKeys != null) { + Arrays.stream(portKeys.split(",")) + .map(String::trim) + .peek(key -> LOG.debug("Adding port key " + key + " to mesos request")) + .forEach(tmPortKeys::add); + } return tmPortKeys; } diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java index 6784e427c1f59..7fc99d28db3d2 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java @@ -52,4 +52,19 @@ public void canGetPortKeys() { assertEquals("port key must be correct", "anotherport", iterator.next()); } + @Test + public void canGetNoPortKeys() { + // Setup + Configuration config = new Configuration(); + + // Act + Set portKeys = LaunchableMesosWorker.extractPortKeys(config); + + // Assert + assertEquals("Must get right number of port keys", 2, portKeys.size()); + Iterator iterator = portKeys.iterator(); + assertEquals("port key must be correct", LaunchableMesosWorker.TM_PORT_KEYS[0], iterator.next()); + assertEquals("port key must be correct", LaunchableMesosWorker.TM_PORT_KEYS[1], iterator.next()); + } + } From e3017db7a54edea22b546d5f427b3f82ade56b25 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 30 Nov 2018 11:17:14 +0100 Subject: [PATCH 179/359] [FLINK-10149][mesos] Replace string concatenation with slf4j placeholders --- .../mesos/runtime/clusterframework/LaunchableMesosWorker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 35bea9918eafe..81ee79524a5b8 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -349,7 +349,7 @@ static Set extractPortKeys(Configuration config) { if (portKeys != null) { Arrays.stream(portKeys.split(",")) .map(String::trim) - .peek(key -> LOG.debug("Adding port key " + key + " to mesos request")) + .peek(key -> LOG.debug("Adding port key {} to mesos request")) .forEach(tmPortKeys::add); } From 3108567f886177cba7ae72ad1dd5124c57987860 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 30 Nov 2018 11:20:18 +0100 Subject: [PATCH 180/359] [FLINK-10149][mesos] Make returned port keys set immutable. --- .../mesos/runtime/clusterframework/LaunchableMesosWorker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 81ee79524a5b8..1eb5679dc8314 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -353,7 +353,7 @@ static Set extractPortKeys(Configuration config) { .forEach(tmPortKeys::add); } - return tmPortKeys; + return Collections.unmodifiableSet(tmPortKeys); } @Override From 0ff5f86f78a71ae77ccbe0e9abf5a387e339f67a Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 30 Nov 2018 11:50:39 +0100 Subject: [PATCH 181/359] [FLINK-10149][mesos] Simplify assertions in LaunchableMesosWorkerTest Use Set equality to simplify test assertions. Change type of field LaunchableMesosWorker#TM_PORT_KEYS to Set. --- .../LaunchableMesosWorker.java | 8 +++--- .../LaunchableMesosWorkerTest.java | 25 +++++++++---------- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 1eb5679dc8314..637442c899d31 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; @@ -64,12 +65,13 @@ public class LaunchableMesosWorker implements LaunchableTask { protected static final Logger LOG = LoggerFactory.getLogger(LaunchableMesosWorker.class); + /** * The set of configuration keys to be dynamically configured with a port allocated from Mesos. */ - static final String[] TM_PORT_KEYS = { + static final Set TM_PORT_KEYS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList( "taskmanager.rpc.port", - "taskmanager.data.port"}; + "taskmanager.data.port"))); private final MesosArtifactResolver resolver; private final ContainerSpecification containerSpec; @@ -342,7 +344,7 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, MesosResourceAllocation al * @return A deterministically ordered Set of port keys to expose from the TM container */ static Set extractPortKeys(Configuration config) { - final LinkedHashSet tmPortKeys = new LinkedHashSet<>(Arrays.asList(TM_PORT_KEYS)); + final LinkedHashSet tmPortKeys = new LinkedHashSet<>(TM_PORT_KEYS); final String portKeys = config.getString(PORT_ASSIGNMENTS); diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java index 7fc99d28db3d2..48a436cb9954f 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java @@ -23,11 +23,14 @@ import org.junit.Test; -import java.util.Iterator; +import java.util.Arrays; +import java.util.HashSet; import java.util.Set; import static org.apache.flink.mesos.configuration.MesosOptions.PORT_ASSIGNMENTS; -import static org.junit.Assert.assertEquals; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; /** * Test that mesos config are extracted correctly from the configuration. @@ -37,19 +40,18 @@ public class LaunchableMesosWorkerTest extends TestLogger { @Test public void canGetPortKeys() { // Setup + Set additionalPorts = new HashSet<>(Arrays.asList("someport.here", "anotherport")); + Configuration config = new Configuration(); - config.setString(PORT_ASSIGNMENTS, "someport.here,anotherport"); + config.setString(PORT_ASSIGNMENTS, String.join(",", additionalPorts)); // Act Set portKeys = LaunchableMesosWorker.extractPortKeys(config); // Assert - assertEquals("Must get right number of port keys", 4, portKeys.size()); - Iterator iterator = portKeys.iterator(); - assertEquals("port key must be correct", LaunchableMesosWorker.TM_PORT_KEYS[0], iterator.next()); - assertEquals("port key must be correct", LaunchableMesosWorker.TM_PORT_KEYS[1], iterator.next()); - assertEquals("port key must be correct", "someport.here", iterator.next()); - assertEquals("port key must be correct", "anotherport", iterator.next()); + Set expectedPorts = new HashSet<>(LaunchableMesosWorker.TM_PORT_KEYS); + expectedPorts.addAll(additionalPorts); + assertThat(portKeys, is(equalTo(expectedPorts))); } @Test @@ -61,10 +63,7 @@ public void canGetNoPortKeys() { Set portKeys = LaunchableMesosWorker.extractPortKeys(config); // Assert - assertEquals("Must get right number of port keys", 2, portKeys.size()); - Iterator iterator = portKeys.iterator(); - assertEquals("port key must be correct", LaunchableMesosWorker.TM_PORT_KEYS[0], iterator.next()); - assertEquals("port key must be correct", LaunchableMesosWorker.TM_PORT_KEYS[1], iterator.next()); + assertThat(portKeys, is(equalTo(LaunchableMesosWorker.TM_PORT_KEYS))); } } From fe1f7cbd0ae4f945dcab3f589bbc51f380aad94d Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 27 Nov 2018 22:02:31 +0800 Subject: [PATCH 182/359] [FLINK-11013] [table] Fix distinct aggregates for group window in Table API This closes #7181 --- .../flink/table/plan/logical/operators.scala | 2 + .../api/stream/table/AggregateTest.scala | 75 +++++++++++++++++++ 2 files changed, 77 insertions(+) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala index 84e3f79cec3e2..c0cfa247bca86 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala @@ -633,6 +633,8 @@ case class WindowAggregate( case aggExpr: Aggregation if aggExpr.getSqlAggFunction.requiresOver => failValidation(s"OVER clause is necessary for window functions: [${aggExpr.getClass}].") + case aggExpr: DistinctAgg => + validateAggregateExpression(aggExpr.child) // check no nested aggregation exists. case aggExpr: Aggregation => aggExpr.children.foreach { child => diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/AggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/AggregateTest.scala index 671f8dd8d1dab..afa9f8b0c7947 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/AggregateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/AggregateTest.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.api.stream.table import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ +import org.apache.flink.table.plan.logical.{SessionGroupWindow, SlidingGroupWindow, TumblingGroupWindow} import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase @@ -238,4 +239,78 @@ class AggregateTest extends TableTestBase { util.verifyTable(resultTable, expected) } + + @Test + def testDistinctAggregateOnTumbleWindow(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Long, String)]( + "MyTable", 'a, 'b, 'c, 'rowtime.rowtime) + val result = table + .window(Tumble over 15.minute on 'rowtime as 'w) + .groupBy('w) + .select('a.count.distinct, 'a.sum) + + val expected = unaryNode( + "DataStreamGroupWindowAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "rowtime") + ), + term("window", TumblingGroupWindow('w, 'rowtime, 900000.millis)), + term("select", "COUNT(DISTINCT a) AS TMP_0", "SUM(a) AS TMP_1") + ) + + util.verifyTable(result, expected) + } + + @Test + def testMultiDistinctAggregateSameFieldOnHopWindow(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Long, String)]( + "MyTable", 'a, 'b, 'c, 'rowtime.rowtime) + val result = table + .window(Slide over 1.hour every 15.minute on 'rowtime as 'w) + .groupBy('w) + .select('a.count.distinct, 'a.sum.distinct, 'a.max.distinct) + + val expected = unaryNode( + "DataStreamGroupWindowAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "rowtime") + ), + term("window", SlidingGroupWindow('w, 'rowtime, 3600000.millis, 900000.millis)), + term("select", "COUNT(DISTINCT a) AS TMP_0", "SUM(DISTINCT a) AS TMP_1", + "MAX(DISTINCT a) AS TMP_2") + ) + + util.verifyTable(result, expected) + } + + @Test + def testDistinctAggregateWithGroupingOnSessionWindow(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Long, String)]( + "MyTable", 'a, 'b, 'c, 'rowtime.rowtime) + val result = table + .window(Session withGap 15.minute on 'rowtime as 'w) + .groupBy('a, 'w) + .select('a, 'a.count, 'c.count.distinct) + + val expected = unaryNode( + "DataStreamGroupWindowAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "a", "c", "rowtime") + ), + term("groupBy", "a"), + term("window", SessionGroupWindow('w, 'rowtime, 900000.millis)), + term("select", "a", "COUNT(a) AS TMP_0", "COUNT(DISTINCT c) AS TMP_1") + ) + + util.verifyTable(result, expected) + } } From 7b23bf69e80aae4d60b1f9f8b5175a25f144930b Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 14 Nov 2018 13:57:44 +0100 Subject: [PATCH 183/359] [FLINK-10874][kafka-docs] Document likely cause of UnknownTopicOrPartitionException --- docs/dev/connectors/kafka.md | 62 ++++++++++++++++++++---------------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 0630c6ec7d6c7..351a4dc2d4126 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -660,19 +660,6 @@ we recommend setting the number of retries to a higher value. **Note**: There is currently no transactional producer for Kafka, so Flink can not guarantee exactly-once delivery into a Kafka topic. -
    - Attention: Depending on your Kafka configuration, even after Kafka acknowledges - writes you can still experience data loss. In particular keep in mind the following Kafka settings: -
      -
    • acks
    • -
    • log.flush.interval.messages
    • -
    • log.flush.interval.ms
    • -
    • log.flush.*
    • -
    - Default values for the above options can easily lead to data loss. Please refer to Kafka documentation - for more explanation. -
    - #### Kafka 0.11 and newer With Flink's checkpointing enabled, the `FlinkKafkaProducer011` (`FlinkKafkaProducer` for Kafka >= 1.0.0 versions) can provide @@ -690,21 +677,6 @@ chosen by passing appropriate `semantic` parameter to the `FlinkKafkaProducer011 or `read_uncommitted` - the latter one is the default value) for any application consuming records from Kafka. -
    - Attention: Depending on your Kafka configuration, even after Kafka acknowledges - writes you can still experience data losses. In particular keep in mind about following properties - in Kafka config: -
      -
    • acks
    • -
    • log.flush.interval.messages
    • -
    • log.flush.interval.ms
    • -
    • log.flush.*
    • -
    - Default values for the above options can easily lead to data loss. Please refer to the Kafka documentation - for more explanation. -
    - - ##### Caveats `Semantic.EXACTLY_ONCE` mode relies on the ability to commit transactions @@ -831,4 +803,38 @@ A mismatch in service name between client and server configuration will cause th For more information on Flink configuration for Kerberos security, please see [here]({{ site.baseurl}}/ops/config.html). You can also find [here]({{ site.baseurl}}/ops/security-kerberos.html) further details on how Flink internally setups Kerberos-based security. +## Troubleshooting + +
    +If you have a problem with Kafka when using Flink, keep in mind that Flink only wraps +KafkaConsumer or +KafkaProducer +and your problem might be independent of Flink and sometimes can be solved by upgrading Kafka brokers, +reconfiguring Kafka brokers or reconfiguring KafkaConsumer or KafkaProducer in Flink. +Some examples of common problems are listed below. +
    + +### Data loss + +Depending on your Kafka configuration, even after Kafka acknowledges +writes you can still experience data loss. In particular keep in mind about the following properties +in Kafka config: + +- `acks` +- `log.flush.interval.messages` +- `log.flush.interval.ms` +- `log.flush.*` + +Default values for the above options can easily lead to data loss. +Please refer to the Kafka documentation for more explanation. + +### UnknownTopicOrPartitionException + +One possible cause of this error is when a new leader election is taking place, +for example after or during restarting a Kafka broker. +This is a retriable exception, so Flink job should be able to restart and resume normal operation. +It also can be circumvented by changing `retries` property in the producer settings. +However this might cause reordering of messages, +which in turn if undesired can be circumvented by setting `max.in.flight.requests.per.connection` to 1. + {% top %} From a7f4acc92084c27b490278b2e027adfe88d03a1f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 3 Dec 2018 11:46:13 +0100 Subject: [PATCH 184/359] [FLINK-11047] Fix CoGroupGroupSortTranslationTest by specyfing types Adding the parameter types is necessary to compile the CoGroupGroupSortTranslationTest with Scala 2.12. --- .../translation/CoGroupGroupSortTranslationTest.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala index bf07aa0437fec..1d571d0b3f415 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CoGroupGroupSortTranslationTest.scala @@ -128,7 +128,11 @@ class CoGroupGroupSortTranslationTest extends TestLogger { .where(1).equalTo(2) .sortFirstGroup(0, Order.DESCENDING) .sortSecondGroup(1, Order.ASCENDING).sortSecondGroup(0, Order.DESCENDING) - .apply((a, b, c: Collector[(Long, Long)]) => a.foreach(e => c.collect(e))) + .apply( + (a: Iterator[(Long, Long)], + b: Iterator[(Long, Long, Long)], + c: Collector[(Long, Long)]) => + a.foreach(e => c.collect(e))) .output(new DiscardingOutputFormat[(Long, Long)]) val p = env.createProgramPlan() From 1034f3e920df88bee75ff36a3d8732a1bd7b2913 Mon Sep 17 00:00:00 2001 From: kgorman Date: Fri, 30 Nov 2018 16:29:41 -0600 Subject: [PATCH 185/359] [FLINK-11044] [docs] Fix registerTableSink docs This closes #7208. --- docs/dev/table/connect.md | 56 +++++++++++++++++++-------------------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index effd913707ea6..d8677714fa86f 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -312,7 +312,7 @@ The following timestamp extractors are supported: .timestampsFromField("ts_field") // required: original field name in the input ) -// Converts the assigned timestamps from a DataStream API record into the rowtime attribute +// Converts the assigned timestamps from a DataStream API record into the rowtime attribute // and thus preserves the assigned timestamps from the source. // This requires a source that assigns timestamps (e.g., Kafka 0.10+). .rowtime( @@ -337,7 +337,7 @@ rowtime: type: from-field from: "ts_field" # required: original field name in the input -# Converts the assigned timestamps from a DataStream API record into the rowtime attribute +# Converts the assigned timestamps from a DataStream API record into the rowtime attribute # and thus preserves the assigned timestamps from the source. rowtime: timestamps: @@ -351,7 +351,7 @@ The following watermark strategies are supported:
    {% highlight java %} -// Sets a watermark strategy for ascending rowtime attributes. Emits a watermark of the maximum +// Sets a watermark strategy for ascending rowtime attributes. Emits a watermark of the maximum // observed timestamp so far minus 1. Rows that have a timestamp equal to the max timestamp // are not late. .rowtime( @@ -377,7 +377,7 @@ The following watermark strategies are supported:
    {% highlight yaml %} -# Sets a watermark strategy for ascending rowtime attributes. Emits a watermark of the maximum +# Sets a watermark strategy for ascending rowtime attributes. Emits a watermark of the maximum # observed timestamp so far minus 1. Rows that have a timestamp equal to the max timestamp # are not late. rowtime: @@ -695,7 +695,7 @@ connector: **Key extraction:** Flink automatically extracts valid keys from a query. For example, a query `SELECT a, b, c FROM t GROUP BY a, b` defines a composite key of the fields `a` and `b`. The Elasticsearch connector generates a document ID string for every row by concatenating all key fields in the order defined in the query using a key delimiter. A custom representation of null literals for key fields can be defined. -Attention A JSON format defines how to encode documents for the external system, therefore, it must be added as a [dependency](connect.html#formats). +Attention A JSON format defines how to encode documents for the external system, therefore, it must be added as a [dependency](connect.html#formats). {% top %} @@ -717,8 +717,8 @@ The CSV format allows to read and write comma-separated rows. new Csv() .field("field1", Types.STRING) // required: ordered format fields .field("field2", Types.TIMESTAMP) - .fieldDelimiter(",") // optional: string delimiter "," by default - .lineDelimiter("\n") // optional: string delimiter "\n" by default + .fieldDelimiter(",") // optional: string delimiter "," by default + .lineDelimiter("\n") // optional: string delimiter "\n" by default .quoteCharacter('"') // optional: single character for string values, empty by default .commentPrefix('#') // optional: string to indicate comments, empty by default .ignoreFirstLine() // optional: ignore the first line, by default it is not skipped @@ -736,8 +736,8 @@ format: type: VARCHAR - name: field2 type: TIMESTAMP - field-delimiter: "," # optional: string delimiter "," by default - line-delimiter: "\n" # optional: string delimiter "\n" by default + field-delimiter: "," # optional: string delimiter "," by default + line-delimiter: "\n" # optional: string delimiter "\n" by default quote-character: '"' # optional: single character for string values, empty by default comment-prefix: '#' # optional: string to indicate comments, empty by default ignore-first-line: false # optional: boolean flag to ignore the first line, by default it is not skipped @@ -992,7 +992,7 @@ These are the additional `TableSink`s which are provided with Flink: | **Class name** | **Maven dependency** | **Batch?** | **Streaming?** | **Description** | `CsvTableSink` | `flink-table` | Y | Append | A simple sink for CSV files. | `JDBCAppendTableSink` | `flink-jdbc` | Y | Append | Writes a Table to a JDBC table. -| `CassandraAppendTableSink` | `flink-connector-cassandra` | N | Append | Writes a Table to a Cassandra table. +| `CassandraAppendTableSink` | `flink-connector-cassandra` | N | Append | Writes a Table to a Cassandra table. ### OrcTableSource @@ -1044,7 +1044,7 @@ val orcTableSource = OrcTableSource.builder() ### CsvTableSink -The `CsvTableSink` emits a `Table` to one or more CSV files. +The `CsvTableSink` emits a `Table` to one or more CSV files. The sink only supports append-only streaming tables. It cannot be used to emit a `Table` that is continuously updated. See the [documentation on Table to Stream conversions](./streaming/dynamic_tables.html#table-to-stream-conversion) for details. When emitting a streaming table, rows are written at least once (if checkpointing is enabled) and the `CsvTableSink` does not split output files into bucket files but continuously writes to the same files. @@ -1053,17 +1053,17 @@ The sink only supports append-only streaming tables. It cannot be used to emit a {% highlight java %} CsvTableSink sink = new CsvTableSink( - path, // output path + path, // output path "|", // optional: delimit files by '|' 1, // optional: write to a single file WriteMode.OVERWRITE); // optional: override existing files tableEnv.registerTableSink( "csvOutputTable", - sink, // specify table schema new String[]{"f0", "f1"}, - new TypeInformation[]{Types.STRING, Types.INT}); + new TypeInformation[]{Types.STRING, Types.INT}, + sink); Table table = ... table.insertInto("csvOutputTable"); @@ -1074,17 +1074,17 @@ table.insertInto("csvOutputTable"); {% highlight scala %} val sink: CsvTableSink = new CsvTableSink( - path, // output path + path, // output path fieldDelim = "|", // optional: delimit files by '|' numFiles = 1, // optional: write to a single file writeMode = WriteMode.OVERWRITE) // optional: override existing files tableEnv.registerTableSink( "csvOutputTable", - sink, // specify table schema Array[String]("f0", "f1"), - Array[TypeInformation[_]](Types.STRING, Types.INT)) + Array[TypeInformation[_]](Types.STRING, Types.INT), + sink) val table: Table = ??? table.insertInto("csvOutputTable") @@ -1113,10 +1113,10 @@ JDBCAppendTableSink sink = JDBCAppendTableSink.builder() tableEnv.registerTableSink( "jdbcOutputTable", - sink, // specify table schema new String[]{"id"}, - new TypeInformation[]{Types.INT}); + new TypeInformation[]{Types.INT}, + sink); Table table = ... table.insertInto("jdbcOutputTable"); @@ -1134,10 +1134,10 @@ val sink: JDBCAppendTableSink = JDBCAppendTableSink.builder() tableEnv.registerTableSink( "jdbcOutputTable", - sink, // specify table schema Array[String]("id"), - Array[TypeInformation[_]](Types.INT)) + Array[TypeInformation[_]](Types.INT), + sink) val table: Table = ??? table.insertInto("jdbcOutputTable") @@ -1145,7 +1145,7 @@ table.insertInto("jdbcOutputTable")
    -Similar to using JDBCOutputFormat, you have to explicitly specify the name of the JDBC driver, the JDBC URL, the query to be executed, and the field types of the JDBC table. +Similar to using JDBCOutputFormat, you have to explicitly specify the name of the JDBC driver, the JDBC URL, the query to be executed, and the field types of the JDBC table. {% top %} @@ -1164,16 +1164,16 @@ To use the `CassandraAppendTableSink`, you have to add the Cassandra connector d ClusterBuilder builder = ... // configure Cassandra cluster connection CassandraAppendTableSink sink = new CassandraAppendTableSink( - builder, + builder, // the query must match the schema of the table INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?)); tableEnv.registerTableSink( "cassandraOutputTable", - sink, // specify table schema new String[]{"id", "name", "value"}, - new TypeInformation[]{Types.INT, Types.STRING, Types.DOUBLE}); + new TypeInformation[]{Types.INT, Types.STRING, Types.DOUBLE}, + sink); Table table = ... table.insertInto(cassandraOutputTable); @@ -1185,16 +1185,16 @@ table.insertInto(cassandraOutputTable); val builder: ClusterBuilder = ... // configure Cassandra cluster connection val sink: CassandraAppendTableSink = new CassandraAppendTableSink( - builder, + builder, // the query must match the schema of the table INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?)) tableEnv.registerTableSink( "cassandraOutputTable", - sink, // specify table schema Array[String]("id", "name", "value"), - Array[TypeInformation[_]](Types.INT, Types.STRING, Types.DOUBLE)) + Array[TypeInformation[_]](Types.INT, Types.STRING, Types.DOUBLE), + sink) val table: Table = ??? table.insertInto(cassandraOutputTable) From 60f21cc235104b11ee61440d092f3986336f4dc3 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 21 Nov 2018 10:35:58 +0100 Subject: [PATCH 186/359] [hotfix][fs-connector] Refactor PartFileWriter to take stream. --- .../api/functions/sink/filesystem/Bucket.java | 7 +++++-- .../api/functions/sink/filesystem/BulkPartWriter.java | 10 ++++------ .../api/functions/sink/filesystem/PartFileWriter.java | 8 ++++---- .../functions/sink/filesystem/RowWisePartWriter.java | 10 ++++------ 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index 65a7628578c39..042bcda31a897 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.util.Preconditions; @@ -124,8 +125,9 @@ private void restoreInProgressFile(final BucketState state) throws IOE // we try to resume the previous in-progress file if (state.hasInProgressResumableFile()) { final RecoverableWriter.ResumeRecoverable resumable = state.getInProgressResumableFile(); + final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable); inProgressPart = partFileFactory.resumeFrom( - bucketId, fsWriter, resumable, state.getInProgressFileCreationTime()); + bucketId, stream, resumable, state.getInProgressFileCreationTime()); } } @@ -195,7 +197,8 @@ private void rollPartFile(final long currentTime) throws IOException { closePartFile(); final Path partFilePath = assembleNewPartPath(); - inProgressPart = partFileFactory.openNew(bucketId, fsWriter, partFilePath, currentTime); + final RecoverableFsDataOutputStream stream = fsWriter.open(partFilePath); + inProgressPart = partFileFactory.openNew(bucketId, stream, partFilePath, currentTime); if (LOG.isDebugEnabled()) { LOG.debug("Subtask {} opening new part file \"{}\" for bucket id={}.", diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java index 005ae4e737fb9..a44b0e8aea3da 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkPartWriter.java @@ -79,14 +79,13 @@ static class Factory implements PartFileWriter.PartFileFactory resumeFrom( final BucketID bucketId, - final RecoverableWriter fileSystemWriter, + final RecoverableFsDataOutputStream stream, final RecoverableWriter.ResumeRecoverable resumable, final long creationTime) throws IOException { - Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(stream); Preconditions.checkNotNull(resumable); - final RecoverableFsDataOutputStream stream = fileSystemWriter.recover(resumable); final BulkWriter writer = writerFactory.create(stream); return new BulkPartWriter<>(bucketId, stream, writer, creationTime); } @@ -94,14 +93,13 @@ public PartFileWriter resumeFrom( @Override public PartFileWriter openNew( final BucketID bucketId, - final RecoverableWriter fileSystemWriter, + final RecoverableFsDataOutputStream stream, final Path path, final long creationTime) throws IOException { - Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(stream); Preconditions.checkNotNull(path); - final RecoverableFsDataOutputStream stream = fileSystemWriter.open(path); final BulkWriter writer = writerFactory.create(stream); return new BulkPartWriter<>(bucketId, stream, writer, creationTime); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java index 662454ba200b3..95a2978a4c601 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileWriter.java @@ -111,7 +111,7 @@ interface PartFileFactory { /** * Used upon recovery from a failure to recover a {@link PartFileWriter writer}. * @param bucketId the id of the bucket this writer is writing to. - * @param fileSystemWriter the filesystem-specific writer to use when writing to the filesystem. + * @param stream the filesystem-specific output stream to use when writing to the filesystem. * @param resumable the state of the stream we are resurrecting. * @param creationTime the creation time of the stream. * @return the recovered {@link PartFileWriter writer}. @@ -119,14 +119,14 @@ interface PartFileFactory { */ PartFileWriter resumeFrom( final BucketID bucketId, - final RecoverableWriter fileSystemWriter, + final RecoverableFsDataOutputStream stream, final RecoverableWriter.ResumeRecoverable resumable, final long creationTime) throws IOException; /** * Used to create a new {@link PartFileWriter writer}. * @param bucketId the id of the bucket this writer is writing to. - * @param fileSystemWriter the filesystem-specific writer to use when writing to the filesystem. + * @param stream the filesystem-specific output stream to use when writing to the filesystem. * @param path the part this writer will write to. * @param creationTime the creation time of the stream. * @return the new {@link PartFileWriter writer}. @@ -134,7 +134,7 @@ PartFileWriter resumeFrom( */ PartFileWriter openNew( final BucketID bucketId, - final RecoverableWriter fileSystemWriter, + final RecoverableFsDataOutputStream stream, final Path path, final long creationTime) throws IOException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java index 2478b79a52797..05c160c262964 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/RowWisePartWriter.java @@ -67,28 +67,26 @@ static class Factory implements PartFileWriter.PartFileFactory resumeFrom( final BucketID bucketId, - final RecoverableWriter fileSystemWriter, + final RecoverableFsDataOutputStream stream, final RecoverableWriter.ResumeRecoverable resumable, final long creationTime) throws IOException { - Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(stream); Preconditions.checkNotNull(resumable); - final RecoverableFsDataOutputStream stream = fileSystemWriter.recover(resumable); return new RowWisePartWriter<>(bucketId, stream, encoder, creationTime); } @Override public PartFileWriter openNew( final BucketID bucketId, - final RecoverableWriter fileSystemWriter, + final RecoverableFsDataOutputStream stream, final Path path, final long creationTime) throws IOException { - Preconditions.checkNotNull(fileSystemWriter); + Preconditions.checkNotNull(stream); Preconditions.checkNotNull(path); - final RecoverableFsDataOutputStream stream = fileSystemWriter.open(path); return new RowWisePartWriter<>(bucketId, stream, encoder, creationTime); } } From 1ef6bf87952861c9643a280f4fe106f94a69ffa8 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 21 Nov 2018 10:43:55 +0100 Subject: [PATCH 187/359] [hotfix][fs-connector] Refactor Bucket to statically import Preconditions. --- .../api/functions/sink/filesystem/Bucket.java | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index 042bcda31a897..8ba35b8be49db 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -23,7 +23,6 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableFsDataOutputStream; import org.apache.flink.core.fs.RecoverableWriter; -import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +37,9 @@ import java.util.Map; import java.util.Objects; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + /** * A bucket is the directory organization of the output of the {@link StreamingFileSink}. * @@ -84,13 +86,13 @@ private Bucket( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy) { - this.fsWriter = Preconditions.checkNotNull(fsWriter); + this.fsWriter = checkNotNull(fsWriter); this.subtaskIndex = subtaskIndex; - this.bucketId = Preconditions.checkNotNull(bucketId); - this.bucketPath = Preconditions.checkNotNull(bucketPath); + this.bucketId = checkNotNull(bucketId); + this.bucketPath = checkNotNull(bucketPath); this.partCounter = initialPartCounter; - this.partFileFactory = Preconditions.checkNotNull(partFileFactory); - this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy); + this.partFileFactory = checkNotNull(partFileFactory); + this.rollingPolicy = checkNotNull(rollingPolicy); this.pendingPartsForCurrentCheckpoint = new ArrayList<>(); this.pendingPartsPerCheckpoint = new HashMap<>(); @@ -158,8 +160,8 @@ boolean isActive() { } void merge(final Bucket bucket) throws IOException { - Preconditions.checkNotNull(bucket); - Preconditions.checkState(Objects.equals(bucket.bucketPath, bucketPath)); + checkNotNull(bucket); + checkState(Objects.equals(bucket.bucketPath, bucketPath)); // There should be no pending files in the "to-merge" states. // The reason is that: @@ -167,8 +169,8 @@ void merge(final Bucket bucket) throws IOException { // So a snapshot, including the one we are recovering from, will never contain such files. // 2) the files in pendingPartsPerCheckpoint are committed upon recovery (see commitRecoveredPendingFiles()). - Preconditions.checkState(bucket.pendingPartsForCurrentCheckpoint.isEmpty()); - Preconditions.checkState(bucket.pendingPartsPerCheckpoint.isEmpty()); + checkState(bucket.pendingPartsForCurrentCheckpoint.isEmpty()); + checkState(bucket.pendingPartsPerCheckpoint.isEmpty()); RecoverableWriter.CommitRecoverable committable = bucket.closePartFile(); if (committable != null) { @@ -257,7 +259,7 @@ private void prepareBucketForCheckpointing(long checkpointId) throws IOException } void onSuccessfulCompletionOfCheckpoint(long checkpointId) throws IOException { - Preconditions.checkNotNull(fsWriter); + checkNotNull(fsWriter); Iterator>> it = pendingPartsPerCheckpoint.entrySet().iterator(); From 3514c52d7539dedac30df2d42fdc05b79f0e4551 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 21 Nov 2018 11:11:10 +0100 Subject: [PATCH 188/359] [hotfix][s3-connector] Renamed S3MultiPartUploader to S3AccessHelper. --- .../common/AbstractS3FileSystemFactory.java | 6 ++-- .../flink/fs/s3/common/FlinkS3FileSystem.java | 12 ++++---- .../RecoverableMultiPartUploadImpl.java | 30 +++++++++---------- ...iPartUploader.java => S3AccessHelper.java} | 2 +- .../fs/s3/common/writer/S3Committer.java | 12 ++++---- .../S3RecoverableMultipartUploadFactory.java | 10 +++---- .../s3/common/writer/S3RecoverableWriter.java | 4 +-- .../fs/s3/common/S3EntropyFsFactoryTest.java | 4 +-- .../RecoverableMultiPartUploadImplTest.java | 4 +-- ...ploader.java => HadoopS3AccessHelper.java} | 26 ++++++++-------- .../fs/s3hadoop/S3FileSystemFactory.java | 6 ++-- .../fs/s3presto/S3FileSystemFactory.java | 4 +-- 12 files changed, 60 insertions(+), 60 deletions(-) rename flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/{S3MultiPartUploader.java => S3AccessHelper.java} (99%) rename flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/{HadoopS3MultiPartUploader.java => HadoopS3AccessHelper.java} (77%) diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java index 318fd39829c79..6ccdeae7df835 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java @@ -25,7 +25,7 @@ import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystemFactory; -import org.apache.flink.fs.s3.common.writer.S3MultiPartUploader; +import org.apache.flink.fs.s3.common.writer.S3AccessHelper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -141,7 +141,7 @@ public FileSystem create(URI fsUri) throws IOException { final String localTmpDirectory = flinkConfig.getString(CoreOptions.TMP_DIRS); final long s3minPartSize = flinkConfig.getLong(PART_UPLOAD_MIN_SIZE); final int maxConcurrentUploads = flinkConfig.getInteger(MAX_CONCURRENT_UPLOADS); - final S3MultiPartUploader s3AccessHelper = getS3AccessHelper(fs); + final S3AccessHelper s3AccessHelper = getS3AccessHelper(fs); return new FlinkS3FileSystem( fs, @@ -166,6 +166,6 @@ protected abstract URI getInitURI( URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig); @Nullable - protected abstract S3MultiPartUploader getS3AccessHelper(org.apache.hadoop.fs.FileSystem fs); + protected abstract S3AccessHelper getS3AccessHelper(org.apache.hadoop.fs.FileSystem fs); } diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java index 553edde75b06c..5248e061a12ec 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java @@ -23,7 +23,7 @@ import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.fs.s3.common.utils.RefCountedFile; import org.apache.flink.fs.s3.common.utils.RefCountedTmpFileCreator; -import org.apache.flink.fs.s3.common.writer.S3MultiPartUploader; +import org.apache.flink.fs.s3.common.writer.S3AccessHelper; import org.apache.flink.fs.s3.common.writer.S3RecoverableWriter; import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; import org.apache.flink.util.Preconditions; @@ -60,7 +60,7 @@ public class FlinkS3FileSystem extends HadoopFileSystem implements EntropyInject private final FunctionWithException tmpFileCreator; @Nullable - private final S3MultiPartUploader s3UploadHelper; + private final S3AccessHelper s3AccessHelper; private final Executor uploadThreadPool; @@ -83,7 +83,7 @@ public FlinkS3FileSystem( String localTmpDirectory, @Nullable String entropyInjectionKey, int entropyLength, - @Nullable S3MultiPartUploader s3UploadHelper, + @Nullable S3AccessHelper s3UploadHelper, long s3uploadPartSize, int maxConcurrentUploadsPerStream) { @@ -99,7 +99,7 @@ public FlinkS3FileSystem( // recoverable writer parameter configuration initialization this.localTmpDir = Preconditions.checkNotNull(localTmpDirectory); this.tmpFileCreator = RefCountedTmpFileCreator.inDirectories(new File(localTmpDirectory)); - this.s3UploadHelper = s3UploadHelper; + this.s3AccessHelper = s3UploadHelper; this.uploadThreadPool = Executors.newCachedThreadPool(); Preconditions.checkArgument(s3uploadPartSize >= S3_MULTIPART_MIN_PART_SIZE); @@ -131,7 +131,7 @@ public String getLocalTmpDir() { @Override public RecoverableWriter createRecoverableWriter() throws IOException { - if (s3UploadHelper == null) { + if (s3AccessHelper == null) { // this is the case for Presto throw new UnsupportedOperationException("This s3 file system implementation does not support recoverable writers."); } @@ -139,7 +139,7 @@ public RecoverableWriter createRecoverableWriter() throws IOException { return S3RecoverableWriter.writer( getHadoopFileSystem(), tmpFileCreator, - s3UploadHelper, + s3AccessHelper, uploadThreadPool, s3uploadPartSize, maxConcurrentUploadsPerStream); diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java index 80042ce22f78b..fe2a4cd2712c2 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java @@ -58,7 +58,7 @@ @NotThreadSafe final class RecoverableMultiPartUploadImpl implements RecoverableMultiPartUpload { - private final S3MultiPartUploader s3MPUploader; + private final S3AccessHelper s3AccessHelper; private final Executor uploadThreadPool; @@ -71,7 +71,7 @@ final class RecoverableMultiPartUploadImpl implements RecoverableMultiPartUpload // ------------------------------------------------------------------------ private RecoverableMultiPartUploadImpl( - S3MultiPartUploader s3uploader, + S3AccessHelper s3AccessHelper, Executor uploadThreadPool, String uploadId, String objectName, @@ -81,7 +81,7 @@ private RecoverableMultiPartUploadImpl( ) { checkArgument(numBytes >= 0L); - this.s3MPUploader = checkNotNull(s3uploader); + this.s3AccessHelper = checkNotNull(s3AccessHelper); this.uploadThreadPool = checkNotNull(uploadThreadPool); this.currentUploadInfo = new MultiPartUploadInfo(objectName, uploadId, partsSoFar, numBytes, incompletePart); this.namePrefixForTempObjects = incompleteObjectNamePrefix(objectName); @@ -111,7 +111,7 @@ public void uploadPart(RefCountedFSOutputStream file) throws IOException { currentUploadInfo.registerNewPart(partLength); file.retain(); // keep the file while the async upload still runs - uploadThreadPool.execute(new UploadTask(s3MPUploader, currentUploadInfo, file, future)); + uploadThreadPool.execute(new UploadTask(s3AccessHelper, currentUploadInfo, file, future)); } @Override @@ -124,7 +124,7 @@ public S3Committer snapshotAndGetCommitter() throws IOException { final S3Recoverable snapshot = snapshotAndGetRecoverable(null); return new S3Committer( - s3MPUploader, + s3AccessHelper, snapshot.getObjectName(), snapshot.uploadId(), snapshot.parts(), @@ -179,7 +179,7 @@ private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream file) th // they do not fall under the user's global TTL on S3. // Figure out a way to clean them. - s3MPUploader.uploadIncompletePart(incompletePartObjectName, inputStream, file.getPos()); + s3AccessHelper.uploadIncompletePart(incompletePartObjectName, inputStream, file.getPos()); } finally { file.release(); @@ -244,14 +244,14 @@ private String createTmpObjectName() { // ------------------------------------------------------------------------ public static RecoverableMultiPartUploadImpl newUpload( - final S3MultiPartUploader s3uploader, + final S3AccessHelper s3AccessHelper, final Executor uploadThreadPool, final String objectName) throws IOException { - final String multiPartUploadId = s3uploader.startMultiPartUpload(objectName); + final String multiPartUploadId = s3AccessHelper.startMultiPartUpload(objectName); return new RecoverableMultiPartUploadImpl( - s3uploader, + s3AccessHelper, uploadThreadPool, multiPartUploadId, objectName, @@ -261,7 +261,7 @@ public static RecoverableMultiPartUploadImpl newUpload( } public static RecoverableMultiPartUploadImpl recoverUpload( - final S3MultiPartUploader s3uploader, + final S3AccessHelper s3AccessHelper, final Executor uploadThreadPool, final String multipartUploadId, final String objectName, @@ -270,7 +270,7 @@ public static RecoverableMultiPartUploadImpl recoverUpload( final Optional incompletePart) { return new RecoverableMultiPartUploadImpl( - s3uploader, + s3AccessHelper, uploadThreadPool, multipartUploadId, objectName, @@ -286,7 +286,7 @@ public static RecoverableMultiPartUploadImpl recoverUpload( private static class UploadTask implements Runnable { - private final S3MultiPartUploader s3uploader; + private final S3AccessHelper s3AccessHelper; private final String objectName; @@ -299,7 +299,7 @@ private static class UploadTask implements Runnable { private final CompletableFuture future; UploadTask( - final S3MultiPartUploader s3uploader, + final S3AccessHelper s3AccessHelper, final MultiPartUploadInfo currentUpload, final RefCountedFSOutputStream file, final CompletableFuture future) { @@ -313,7 +313,7 @@ private static class UploadTask implements Runnable { // these are limits put by Amazon checkArgument(partNumber >= 1 && partNumber <= 10_000); - this.s3uploader = checkNotNull(s3uploader); + this.s3AccessHelper = checkNotNull(s3AccessHelper); this.file = checkNotNull(file); this.future = checkNotNull(future); } @@ -321,7 +321,7 @@ private static class UploadTask implements Runnable { @Override public void run() { try (final InputStream inputStream = file.getInputStream()) { - final UploadPartResult result = s3uploader.uploadPart(objectName, uploadId, partNumber, inputStream, file.getPos()); + final UploadPartResult result = s3AccessHelper.uploadPart(objectName, uploadId, partNumber, inputStream, file.getPos()); future.complete(new PartETag(result.getPartNumber(), result.getETag())); file.release(); } diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3MultiPartUploader.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java similarity index 99% rename from flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3MultiPartUploader.java rename to flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java index da227a47488e6..57920a524993f 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3MultiPartUploader.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java @@ -41,7 +41,7 @@ * the upload with all its parts will be either committed or discarded. */ @Internal -public interface S3MultiPartUploader { +public interface S3AccessHelper { /** * Initializes a Multi-Part Upload. diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3Committer.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3Committer.java index 1fc8bf1eaa8dd..5fbc5bb5d42de 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3Committer.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3Committer.java @@ -40,7 +40,7 @@ public final class S3Committer implements RecoverableFsDataOutputStream.Committe private static final Logger LOG = LoggerFactory.getLogger(S3Committer.class); - private final S3MultiPartUploader s3uploader; + private final S3AccessHelper s3AccessHelper; private final String uploadId; @@ -50,8 +50,8 @@ public final class S3Committer implements RecoverableFsDataOutputStream.Committe private final long totalLength; - S3Committer(S3MultiPartUploader s3uploader, String objectName, String uploadId, List parts, long totalLength) { - this.s3uploader = checkNotNull(s3uploader); + S3Committer(S3AccessHelper s3AccessHelper, String objectName, String uploadId, List parts, long totalLength) { + this.s3AccessHelper = checkNotNull(s3AccessHelper); this.objectName = checkNotNull(objectName); this.uploadId = checkNotNull(uploadId); this.parts = checkNotNull(parts); @@ -64,7 +64,7 @@ public void commit() throws IOException { LOG.info("Committing {} with MPU ID {}", objectName, uploadId); final AtomicInteger errorCount = new AtomicInteger(); - s3uploader.commitMultiPartUpload(objectName, uploadId, parts, totalLength, errorCount); + s3AccessHelper.commitMultiPartUpload(objectName, uploadId, parts, totalLength, errorCount); if (errorCount.get() == 0) { LOG.debug("Successfully committed {} with MPU ID {}", objectName, uploadId); @@ -82,14 +82,14 @@ public void commitAfterRecovery() throws IOException { LOG.info("Trying to commit after recovery {} with MPU ID {}", objectName, uploadId); try { - s3uploader.commitMultiPartUpload(objectName, uploadId, parts, totalLength, new AtomicInteger()); + s3AccessHelper.commitMultiPartUpload(objectName, uploadId, parts, totalLength, new AtomicInteger()); } catch (IOException e) { LOG.info("Failed to commit after recovery {} with MPU ID {}. " + "Checking if file was committed before...", objectName, uploadId); LOG.trace("Exception when committing:", e); try { - ObjectMetadata metadata = s3uploader.getObjectMetadata(objectName); + ObjectMetadata metadata = s3AccessHelper.getObjectMetadata(objectName); if (totalLength != metadata.getContentLength()) { String message = String.format("Inconsistent result for object %s: conflicting lengths. " + "Recovered committer for upload %s indicates %s bytes, present object is %s bytes", diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java index b201981f31a04..9a171ae3ebdbc 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java @@ -43,7 +43,7 @@ final class S3RecoverableMultipartUploadFactory { private final org.apache.hadoop.fs.FileSystem fs; - private final S3MultiPartUploader twoPhaseUploader; + private final S3AccessHelper s3AccessHelper; private final FunctionWithException tmpFileSupplier; @@ -53,7 +53,7 @@ final class S3RecoverableMultipartUploadFactory { S3RecoverableMultipartUploadFactory( final FileSystem fs, - final S3MultiPartUploader twoPhaseUploader, + final S3AccessHelper s3AccessHelper, final int maxConcurrentUploadsPerStream, final Executor executor, final FunctionWithException tmpFileSupplier) { @@ -61,14 +61,14 @@ final class S3RecoverableMultipartUploadFactory { this.fs = Preconditions.checkNotNull(fs); this.maxConcurrentUploadsPerStream = maxConcurrentUploadsPerStream; this.executor = executor; - this.twoPhaseUploader = twoPhaseUploader; + this.s3AccessHelper = s3AccessHelper; this.tmpFileSupplier = tmpFileSupplier; } RecoverableMultiPartUpload getNewRecoverableUpload(Path path) throws IOException { return RecoverableMultiPartUploadImpl.newUpload( - twoPhaseUploader, + s3AccessHelper, limitedExecutor(), pathToObjectName(path)); } @@ -77,7 +77,7 @@ RecoverableMultiPartUpload recoverRecoverableUpload(S3Recoverable recoverable) t final Optional incompletePart = downloadLastDataChunk(recoverable); return RecoverableMultiPartUploadImpl.recoverUpload( - twoPhaseUploader, + s3AccessHelper, limitedExecutor(), recoverable.uploadId(), recoverable.getObjectName(), diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java index 2a84308d5b01d..698f65f7c7214 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java @@ -129,7 +129,7 @@ private static S3Recoverable castToS3Recoverable(RecoverableWriter.CommitRecover public static S3RecoverableWriter writer( final FileSystem fs, final FunctionWithException tempFileCreator, - final S3MultiPartUploader twoPhaseUploader, + final S3AccessHelper s3AccessHelper, final Executor uploadThreadPool, final long userDefinedMinPartSize, final int maxConcurrentUploadsPerStream) { @@ -139,7 +139,7 @@ public static S3RecoverableWriter writer( final S3RecoverableMultipartUploadFactory uploadFactory = new S3RecoverableMultipartUploadFactory( fs, - twoPhaseUploader, + s3AccessHelper, maxConcurrentUploadsPerStream, uploadThreadPool, tempFileCreator); diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java index d3d25c347e8ca..5b15652db62f9 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java @@ -19,7 +19,7 @@ package org.apache.flink.fs.s3.common; import org.apache.flink.configuration.Configuration; -import org.apache.flink.fs.s3.common.writer.S3MultiPartUploader; +import org.apache.flink.fs.s3.common.writer.S3AccessHelper; import org.apache.flink.util.TestLogger; import org.apache.hadoop.fs.FileSystem; @@ -78,7 +78,7 @@ protected URI getInitURI(URI fsUri, org.apache.hadoop.conf.Configuration hadoopC @Nullable @Override - protected S3MultiPartUploader getS3AccessHelper(FileSystem fs) { + protected S3AccessHelper getS3AccessHelper(FileSystem fs) { return null; } diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java index 72554e197739f..4c2f1474cc9ba 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java @@ -339,10 +339,10 @@ public void execute(Runnable command) { } /** - * A {@link S3MultiPartUploader} that simulates uploading part files to S3 by + * A {@link S3AccessHelper} that simulates uploading part files to S3 by * simply putting complete and incomplete part files in lists for further validation. */ - private static class StubMultiPartUploader implements S3MultiPartUploader { + private static class StubMultiPartUploader implements S3AccessHelper { private final List completePartsUploaded = new ArrayList<>(); private final List incompletePartsUploaded = new ArrayList<>(); diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3MultiPartUploader.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java similarity index 77% rename from flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3MultiPartUploader.java rename to flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java index f446f70e2a702..f833471b2f031 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3MultiPartUploader.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java @@ -18,7 +18,7 @@ package org.apache.flink.fs.s3hadoop; -import org.apache.flink.fs.s3.common.writer.S3MultiPartUploader; +import org.apache.flink.fs.s3.common.writer.S3AccessHelper; import org.apache.flink.util.MathUtils; import com.amazonaws.SdkBaseException; @@ -43,16 +43,16 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * An implementation of the {@link S3MultiPartUploader} for the Hadoop S3A filesystem. + * An implementation of the {@link S3AccessHelper} for the Hadoop S3A filesystem. */ -public class HadoopS3MultiPartUploader implements S3MultiPartUploader { +public class HadoopS3AccessHelper implements S3AccessHelper { private final S3AFileSystem s3a; - private final InternalWriteOperationHelper s3uploader; + private final InternalWriteOperationHelper s3accessHelper; - public HadoopS3MultiPartUploader(S3AFileSystem s3a, Configuration conf) { - this.s3uploader = new InternalWriteOperationHelper( + public HadoopS3AccessHelper(S3AFileSystem s3a, Configuration conf) { + this.s3accessHelper = new InternalWriteOperationHelper( checkNotNull(s3a), checkNotNull(conf) ); @@ -61,25 +61,25 @@ public HadoopS3MultiPartUploader(S3AFileSystem s3a, Configuration conf) { @Override public String startMultiPartUpload(String key) throws IOException { - return s3uploader.initiateMultiPartUpload(key); + return s3accessHelper.initiateMultiPartUpload(key); } @Override public UploadPartResult uploadPart(String key, String uploadId, int partNumber, InputStream inputStream, long length) throws IOException { - final UploadPartRequest uploadRequest = s3uploader.newUploadPartRequest( + final UploadPartRequest uploadRequest = s3accessHelper.newUploadPartRequest( key, uploadId, partNumber, MathUtils.checkedDownCast(length), inputStream, null, 0L); - return s3uploader.uploadPart(uploadRequest); + return s3accessHelper.uploadPart(uploadRequest); } @Override public PutObjectResult uploadIncompletePart(String key, InputStream inputStream, long length) throws IOException { - final PutObjectRequest putRequest = s3uploader.createPutObjectRequest(key, inputStream, length); - return s3uploader.putObject(putRequest); + final PutObjectRequest putRequest = s3accessHelper.createPutObjectRequest(key, inputStream, length); + return s3accessHelper.putObject(putRequest); } @Override public CompleteMultipartUploadResult commitMultiPartUpload(String destKey, String uploadId, List partETags, long length, AtomicInteger errorCount) throws IOException { - return s3uploader.completeMPUwithRetries(destKey, uploadId, partETags, length, errorCount); + return s3accessHelper.completeMPUwithRetries(destKey, uploadId, partETags, length, errorCount); } @Override @@ -94,7 +94,7 @@ public ObjectMetadata getObjectMetadata(String key) throws IOException { /** * Internal {@link WriteOperationHelper} that is wrapped so that it only exposes - * the functionality we need for the {@link S3MultiPartUploader}. + * the functionality we need for the {@link S3AccessHelper}. */ private static final class InternalWriteOperationHelper extends WriteOperationHelper { diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java index 897629f791039..2637e7b2e23cb 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/S3FileSystemFactory.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.fs.s3.common.AbstractS3FileSystemFactory; import org.apache.flink.fs.s3.common.HadoopConfigLoader; -import org.apache.flink.fs.s3.common.writer.S3MultiPartUploader; +import org.apache.flink.fs.s3.common.writer.S3AccessHelper; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.s3a.S3AFileSystem; @@ -96,8 +96,8 @@ else if (scheme != null && authority == null) { @Nullable @Override - protected S3MultiPartUploader getS3AccessHelper(FileSystem fs) { + protected S3AccessHelper getS3AccessHelper(FileSystem fs) { final S3AFileSystem s3Afs = (S3AFileSystem) fs; - return new HadoopS3MultiPartUploader(s3Afs, s3Afs.getConf()); + return new HadoopS3AccessHelper(s3Afs, s3Afs.getConf()); } } diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java index b579d6ebf0496..0fb28571960c0 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java +++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.fs.s3.common.AbstractS3FileSystemFactory; import org.apache.flink.fs.s3.common.HadoopConfigLoader; -import org.apache.flink.fs.s3.common.writer.S3MultiPartUploader; +import org.apache.flink.fs.s3.common.writer.S3AccessHelper; import org.apache.flink.util.FlinkRuntimeException; import com.facebook.presto.hive.PrestoS3FileSystem; @@ -92,7 +92,7 @@ else if (scheme != null && authority == null) { @Nullable @Override - protected S3MultiPartUploader getS3AccessHelper(FileSystem fs) { + protected S3AccessHelper getS3AccessHelper(FileSystem fs) { return null; } From a029f3363b3b15caeba608c88ebafa2d8c8edd87 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 22 Nov 2018 10:46:10 +0100 Subject: [PATCH 189/359] [hotfix] Consolidated all S3 accesses under the S3AccessHelper. --- .../RecoverableMultiPartUploadImpl.java | 2 +- .../fs/s3/common/writer/S3AccessHelper.java | 21 ++++++++--- .../S3RecoverableMultipartUploadFactory.java | 37 +++++-------------- .../RecoverableMultiPartUploadImplTest.java | 11 ++++-- .../fs/s3hadoop/HadoopS3AccessHelper.java | 25 ++++++++++++- 5 files changed, 58 insertions(+), 38 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java index fe2a4cd2712c2..9f0a8110d555e 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java @@ -179,7 +179,7 @@ private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream file) th // they do not fall under the user's global TTL on S3. // Figure out a way to clean them. - s3AccessHelper.uploadIncompletePart(incompletePartObjectName, inputStream, file.getPos()); + s3AccessHelper.putObject(incompletePartObjectName, inputStream, file.getPos()); } finally { file.release(); diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java index 57920a524993f..dbc099a7b8010 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java @@ -26,6 +26,7 @@ import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.UploadPartResult; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.util.List; @@ -66,10 +67,9 @@ public interface S3AccessHelper { UploadPartResult uploadPart(String key, String uploadId, int partNumber, InputStream file, long length) throws IOException; /** - * Uploads a part and associates it with the MPU with the provided {@code uploadId}. - * - *

    Contrary to the {@link #uploadIncompletePart(String, InputStream, long)}, this part can - * be smaller than the minimum part size imposed by S3. + * Uploads an object to S3. Contrary to the {@link #uploadPart(String, String, int, InputStream, long)} method, + * this object is not going to be associated to any MPU and, as such, it is not subject to the garbage collection + * policies specified for your S3 bucket. * * @param key the key used to identify this part. * @param file the (local) file holding the data to be uploaded. @@ -77,7 +77,7 @@ public interface S3AccessHelper { * @return The {@link PutObjectResult result} of the attempt to stage the incomplete part. * @throws IOException */ - PutObjectResult uploadIncompletePart(String key, InputStream file, long length) throws IOException; + PutObjectResult putObject(String key, InputStream file, long length) throws IOException; /** * Finalizes a Multi-Part Upload. @@ -92,6 +92,17 @@ public interface S3AccessHelper { */ CompleteMultipartUploadResult commitMultiPartUpload(String key, String uploadId, List partETags, long length, AtomicInteger errorCount) throws IOException; + /** + * Gets the object associated with the provided {@code key} from S3 and + * puts it in the provided {@code targetLocation}. + * + * @param key the key of the object to fetch. + * @param targetLocation the file to read the object to. + * @return The number of bytes read. + * @throws IOException + */ + long getObject(String key, File targetLocation) throws IOException; + /** * Fetches the metadata associated with a given key on S3. * diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java index 9a171ae3ebdbc..ddb09abdb549f 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java @@ -19,10 +19,8 @@ package org.apache.flink.fs.s3.common.writer; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.fs.Path; import org.apache.flink.fs.s3.common.utils.BackPressuringExecutor; -import org.apache.flink.fs.s3.common.utils.OffsetAwareOutputStream; import org.apache.flink.fs.s3.common.utils.RefCountedFile; import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; import org.apache.flink.util.Preconditions; @@ -74,7 +72,7 @@ RecoverableMultiPartUpload getNewRecoverableUpload(Path path) throws IOException } RecoverableMultiPartUpload recoverRecoverableUpload(S3Recoverable recoverable) throws IOException { - final Optional incompletePart = downloadLastDataChunk(recoverable); + final Optional incompletePart = recoverInProgressPart(recoverable); return RecoverableMultiPartUploadImpl.recoverUpload( s3AccessHelper, @@ -86,36 +84,20 @@ RecoverableMultiPartUpload recoverRecoverableUpload(S3Recoverable recoverable) t incompletePart); } - @VisibleForTesting - Optional downloadLastDataChunk(S3Recoverable recoverable) throws IOException { + private Optional recoverInProgressPart(S3Recoverable recoverable) throws IOException { - final String objectName = recoverable.incompleteObjectName(); - if (objectName == null) { + final String objectKey = recoverable.incompleteObjectName(); + if (objectKey == null) { return Optional.empty(); } // download the file (simple way) - final RefCountedFile fileAndStream = tmpFileSupplier.apply(null); - final File file = fileAndStream.getFile(); - - long numBytes = 0L; - - try ( - final OffsetAwareOutputStream outStream = fileAndStream.getStream(); - final org.apache.hadoop.fs.FSDataInputStream inStream = - fs.open(new org.apache.hadoop.fs.Path('/' + objectName)) - ) { - final byte[] buffer = new byte[32 * 1024]; - - int numRead; - while ((numRead = inStream.read(buffer)) > 0) { - outStream.write(buffer, 0, numRead); - numBytes += numRead; - } - } + final RefCountedFile refCountedFile = tmpFileSupplier.apply(null); + final File file = refCountedFile.getFile(); + final long numBytes = s3AccessHelper.getObject(objectKey, file); // some sanity checks - if (numBytes != file.length() || numBytes != fileAndStream.getStream().getLength()) { + if (numBytes != file.length()) { throw new IOException(String.format("Error recovering writer: " + "Downloading the last data chunk file gives incorrect length. " + "File=%d bytes, Stream=%d bytes", @@ -132,8 +114,7 @@ Optional downloadLastDataChunk(S3Recoverable recoverable) throws IOExcepti return Optional.of(file); } - @VisibleForTesting - String pathToObjectName(final Path path) { + private String pathToObjectName(final Path path) { org.apache.hadoop.fs.Path hadoopPath = HadoopFileSystem.toHadoopPath(path); if (!hadoopPath.isAbsolute()) { hadoopPath = new org.apache.hadoop.fs.Path(fs.getWorkingDirectory(), hadoopPath); diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java index 4c2f1474cc9ba..a986111125c22 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java @@ -347,11 +347,11 @@ private static class StubMultiPartUploader implements S3AccessHelper { private final List completePartsUploaded = new ArrayList<>(); private final List incompletePartsUploaded = new ArrayList<>(); - public List getCompletePartsUploaded() { + List getCompletePartsUploaded() { return completePartsUploaded; } - public List getIncompletePartsUploaded() { + List getIncompletePartsUploaded() { return incompletePartsUploaded; } @@ -367,11 +367,16 @@ public UploadPartResult uploadPart(String key, String uploadId, int partNumber, } @Override - public PutObjectResult uploadIncompletePart(String key, InputStream file, long length) throws IOException { + public PutObjectResult putObject(String key, InputStream file, long length) throws IOException { final byte[] content = getFileContentBytes(file, MathUtils.checkedDownCast(length)); return storeAndGetPutObjectResult(key, content); } + @Override + public long getObject(String key, File targetLocation) throws IOException { + return 0; + } + @Override public CompleteMultipartUploadResult commitMultiPartUpload( String key, diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java index f833471b2f031..473439c05530e 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java @@ -35,8 +35,11 @@ import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -72,7 +75,7 @@ public UploadPartResult uploadPart(String key, String uploadId, int partNumber, } @Override - public PutObjectResult uploadIncompletePart(String key, InputStream inputStream, long length) throws IOException { + public PutObjectResult putObject(String key, InputStream inputStream, long length) throws IOException { final PutObjectRequest putRequest = s3accessHelper.createPutObjectRequest(key, inputStream, length); return s3accessHelper.putObject(putRequest); } @@ -82,6 +85,26 @@ public CompleteMultipartUploadResult commitMultiPartUpload(String destKey, Strin return s3accessHelper.completeMPUwithRetries(destKey, uploadId, partETags, length, errorCount); } + @Override + public long getObject(String key, File targetLocation) throws IOException { + long numBytes = 0L; + try ( + final OutputStream outStream = new FileOutputStream(targetLocation); + final org.apache.hadoop.fs.FSDataInputStream inStream = + s3a.open(new org.apache.hadoop.fs.Path('/' + key)) + ) { + final byte[] buffer = new byte[32 * 1024]; + + int numRead; + while ((numRead = inStream.read(buffer)) > 0) { + outStream.write(buffer, 0, numRead); + numBytes += numRead; + } + } + + return numBytes; + } + @Override public ObjectMetadata getObjectMetadata(String key) throws IOException { try { From e32e2338a66ef73e9f237315119325cf8c18fb94 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 22 Nov 2018 10:54:05 +0100 Subject: [PATCH 190/359] [hotfix] Method renaming in the RecoverableMultiPartUploadImpl. --- .../writer/RecoverableMultiPartUploadImpl.java | 14 +++++++------- .../s3/common/writer/IncompletePartPrefixTest.java | 12 ++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java index 9f0a8110d555e..787f28626026c 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java @@ -84,7 +84,7 @@ private RecoverableMultiPartUploadImpl( this.s3AccessHelper = checkNotNull(s3AccessHelper); this.uploadThreadPool = checkNotNull(uploadThreadPool); this.currentUploadInfo = new MultiPartUploadInfo(objectName, uploadId, partsSoFar, numBytes, incompletePart); - this.namePrefixForTempObjects = incompleteObjectNamePrefix(objectName); + this.namePrefixForTempObjects = createIncompletePartObjectNamePrefix(objectName); this.uploadsInProgress = new ArrayDeque<>(); } @@ -171,7 +171,7 @@ private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream file) th } // first, upload the trailing data file. during that time, other in-progress uploads may complete. - final String incompletePartObjectName = createTmpObjectName(); + final String incompletePartObjectName = createIncompletePartObjectName(); file.retain(); try (InputStream inputStream = file.getInputStream()) { @@ -192,7 +192,7 @@ private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream file) th // ------------------------------------------------------------------------ @VisibleForTesting - static String incompleteObjectNamePrefix(String objectName) { + static String createIncompletePartObjectNamePrefix(String objectName) { checkNotNull(objectName); final int lastSlash = objectName.lastIndexOf('/'); @@ -209,6 +209,10 @@ static String incompleteObjectNamePrefix(String objectName) { return parent + (child.isEmpty() ? "" : '_') + child + "_tmp_"; } + private String createIncompletePartObjectName() { + return namePrefixForTempObjects + UUID.randomUUID().toString(); + } + private void awaitPendingPartsUpload() throws IOException { checkState(currentUploadInfo.getRemainingParts() == uploadsInProgress.size()); @@ -235,10 +239,6 @@ private PartETag awaitPendingPartUploadToComplete(CompletableFuture up return completedUploadEtag; } - private String createTmpObjectName() { - return namePrefixForTempObjects + UUID.randomUUID().toString(); - } - // ------------------------------------------------------------------------ // factory methods // ------------------------------------------------------------------------ diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/IncompletePartPrefixTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/IncompletePartPrefixTest.java index a3164f18a16d2..9cee040c067fd 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/IncompletePartPrefixTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/IncompletePartPrefixTest.java @@ -22,36 +22,36 @@ import org.junit.Test; /** - * Tests for the {@link RecoverableMultiPartUploadImpl#incompleteObjectNamePrefix(String)}. + * Tests for the {@link RecoverableMultiPartUploadImpl#createIncompletePartObjectNamePrefix(String)}. */ public class IncompletePartPrefixTest { @Test(expected = NullPointerException.class) public void nullObjectNameShouldThroughException() { - RecoverableMultiPartUploadImpl.incompleteObjectNamePrefix(null); + RecoverableMultiPartUploadImpl.createIncompletePartObjectNamePrefix(null); } @Test public void emptyInitialNameShouldSucceed() { - String objectNamePrefix = RecoverableMultiPartUploadImpl.incompleteObjectNamePrefix(""); + String objectNamePrefix = RecoverableMultiPartUploadImpl.createIncompletePartObjectNamePrefix(""); Assert.assertEquals("_tmp_", objectNamePrefix); } @Test public void nameWithoutSlashShouldSucceed() { - String objectNamePrefix = RecoverableMultiPartUploadImpl.incompleteObjectNamePrefix("no_slash_path"); + String objectNamePrefix = RecoverableMultiPartUploadImpl.createIncompletePartObjectNamePrefix("no_slash_path"); Assert.assertEquals("_no_slash_path_tmp_", objectNamePrefix); } @Test public void nameWithOnlySlashShouldSucceed() { - String objectNamePrefix = RecoverableMultiPartUploadImpl.incompleteObjectNamePrefix("/"); + String objectNamePrefix = RecoverableMultiPartUploadImpl.createIncompletePartObjectNamePrefix("/"); Assert.assertEquals("/_tmp_", objectNamePrefix); } @Test public void normalPathShouldSucceed() { - String objectNamePrefix = RecoverableMultiPartUploadImpl.incompleteObjectNamePrefix("/root/home/test-file"); + String objectNamePrefix = RecoverableMultiPartUploadImpl.createIncompletePartObjectNamePrefix("/root/home/test-file"); Assert.assertEquals("/root/home/_test-file_tmp_", objectNamePrefix); } } From 5de507835b2b9a93376820b79a435b8efe53b8a6 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 22 Nov 2018 11:03:13 +0100 Subject: [PATCH 191/359] [FLINK-10963][fs-connector, s3] Cleanup tmp S3 objects uploaded as backups of in-progress files. --- .../flink/core/fs/RecoverableWriter.java | 28 ++ .../core/fs/local/LocalRecoverableWriter.java | 10 + .../fs/hdfs/HadoopRecoverableWriter.java | 10 + .../RecoverableMultiPartUploadImpl.java | 5 - .../fs/s3/common/writer/S3AccessHelper.java | 10 + .../S3RecoverableMultipartUploadFactory.java | 8 - .../s3/common/writer/S3RecoverableWriter.java | 34 ++- .../RecoverableMultiPartUploadImplTest.java | 5 + .../fs/s3hadoop/HadoopS3AccessHelper.java | 15 +- .../HadoopS3RecoverableWriterITCase.java | 46 +++ .../api/functions/sink/filesystem/Bucket.java | 87 ++++-- .../functions/sink/filesystem/BucketTest.java | 263 ++++++++++++++++++ 12 files changed, 473 insertions(+), 48 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/RecoverableWriter.java b/flink-core/src/main/java/org/apache/flink/core/fs/RecoverableWriter.java index e5bfdb84a02f2..7d54b11bbbc82 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/RecoverableWriter.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/RecoverableWriter.java @@ -121,6 +121,34 @@ public interface RecoverableWriter { */ RecoverableFsDataOutputStream recover(ResumeRecoverable resumable) throws IOException; + /** + * Marks if the writer requires to do any additional cleanup/freeing of resources occupied + * as part of a {@link ResumeRecoverable}, e.g. temporarily files created or objects uploaded + * to external systems. + * + *

    In case cleanup is required, then {@link #cleanupRecoverableState(ResumeRecoverable)} should + * be called. + * + * @return {@code true} if cleanup is required, {@code false} otherwise. + */ + boolean requiresCleanupOfRecoverableState(); + + /** + * Frees up any resources that were previously occupied in order to be able to + * recover from a (potential) failure. These can be temporary files that were written + * to the filesystem or objects that were uploaded to S3. + * + *

    NOTE: This operation should not throw an exception if the resumable has already + * been cleaned up and the resources have been freed. But the contract is that it will throw + * an {@link UnsupportedOperationException} if it is called for a {@code RecoverableWriter} + * whose {@link #requiresCleanupOfRecoverableState()} returns {@code false}. + * + * @param resumable The {@link ResumeRecoverable} whose state we want to clean-up. + * @return {@code true} if the resources were successfully freed, {@code false} otherwise + * (e.g. the file to be deleted was not there for any reason - already deleted or never created). + */ + boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException; + /** * Recovers a recoverable stream consistently at the point indicated by the given CommitRecoverable * for finalizing and committing. This will publish the target file with exactly the data diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableWriter.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableWriter.java index a2f0485a003b6..a43e0b6b6bf4e 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableWriter.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalRecoverableWriter.java @@ -70,6 +70,16 @@ public RecoverableFsDataOutputStream recover(ResumeRecoverable recoverable) thro } } + @Override + public boolean requiresCleanupOfRecoverableState() { + return false; + } + + @Override + public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { + throw new UnsupportedOperationException(); + } + @Override public Committer recoverForCommit(CommitRecoverable recoverable) throws IOException { if (recoverable instanceof LocalRecoverable) { diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java index 305f8ee5612ef..03d741b4b82b5 100644 --- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java +++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java @@ -77,6 +77,16 @@ public RecoverableFsDataOutputStream recover(ResumeRecoverable recoverable) thro } } + @Override + public boolean requiresCleanupOfRecoverableState() { + return false; + } + + @Override + public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { + throw new UnsupportedOperationException(); + } + @Override public Committer recoverForCommit(CommitRecoverable recoverable) throws IOException { if (recoverable instanceof HadoopFsRecoverable) { diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java index 787f28626026c..9d88e65f64d75 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java @@ -174,11 +174,6 @@ private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream file) th final String incompletePartObjectName = createIncompletePartObjectName(); file.retain(); try (InputStream inputStream = file.getInputStream()) { - - // TODO: staged incomplete parts are not cleaned up as - // they do not fall under the user's global TTL on S3. - // Figure out a way to clean them. - s3AccessHelper.putObject(incompletePartObjectName, inputStream, file.getPos()); } finally { diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java index dbc099a7b8010..bcdea3c00b060 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java @@ -92,6 +92,16 @@ public interface S3AccessHelper { */ CompleteMultipartUploadResult commitMultiPartUpload(String key, String uploadId, List partETags, long length, AtomicInteger errorCount) throws IOException; + /** + * Deletes the object associated with the provided key. + * + * @param key The key to be deleted. + * @return {@code true} if the resources were successfully freed, {@code false} otherwise + * (e.g. the file to be deleted was not there). + * @throws IOException + */ + boolean deleteObject(String key) throws IOException; + /** * Gets the object associated with the provided {@code key} from S3 and * puts it in the provided {@code targetLocation}. diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java index ddb09abdb549f..3727e25790437 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableMultipartUploadFactory.java @@ -96,14 +96,6 @@ private Optional recoverInProgressPart(S3Recoverable recoverable) throws I final File file = refCountedFile.getFile(); final long numBytes = s3AccessHelper.getObject(objectKey, file); - // some sanity checks - if (numBytes != file.length()) { - throw new IOException(String.format("Error recovering writer: " + - "Downloading the last data chunk file gives incorrect length. " + - "File=%d bytes, Stream=%d bytes", - file.length(), numBytes)); - } - if (numBytes != recoverable.incompleteObjectLength()) { throw new IOException(String.format("Error recovering writer: " + "Downloading the last data chunk file gives incorrect length." + diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java index 698f65f7c7214..ddb4443c58564 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3RecoverableWriter.java @@ -26,7 +26,6 @@ import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.fs.s3.common.utils.RefCountedFile; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.FunctionWithException; import org.apache.hadoop.fs.FileSystem; @@ -37,6 +36,7 @@ import static org.apache.flink.fs.s3.common.FlinkS3FileSystem.S3_MULTIPART_MIN_PART_SIZE; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * An implementation of the {@link RecoverableWriter} against S3. @@ -54,16 +54,20 @@ public class S3RecoverableWriter implements RecoverableWriter { private final long userDefinedMinPartSize; + private final S3AccessHelper s3AccessHelper; + private final S3RecoverableMultipartUploadFactory uploadFactory; @VisibleForTesting S3RecoverableWriter( + final S3AccessHelper s3AccessHelper, final S3RecoverableMultipartUploadFactory uploadFactory, final FunctionWithException tempFileCreator, final long userDefinedMinPartSize) { - this.uploadFactory = Preconditions.checkNotNull(uploadFactory); - this.tempFileCreator = Preconditions.checkNotNull(tempFileCreator); + this.s3AccessHelper = checkNotNull(s3AccessHelper); + this.uploadFactory = checkNotNull(uploadFactory); + this.tempFileCreator = checkNotNull(tempFileCreator); this.userDefinedMinPartSize = userDefinedMinPartSize; } @@ -78,14 +82,14 @@ public RecoverableFsDataOutputStream open(Path path) throws IOException { } @Override - public Committer recoverForCommit(RecoverableWriter.CommitRecoverable recoverable) throws IOException { + public Committer recoverForCommit(CommitRecoverable recoverable) throws IOException { final S3Recoverable s3recoverable = castToS3Recoverable(recoverable); final S3RecoverableFsDataOutputStream recovered = recover(s3recoverable); return recovered.closeForCommit(); } @Override - public S3RecoverableFsDataOutputStream recover(RecoverableWriter.ResumeRecoverable recoverable) throws IOException { + public S3RecoverableFsDataOutputStream recover(ResumeRecoverable recoverable) throws IOException { final S3Recoverable s3recoverable = castToS3Recoverable(recoverable); final RecoverableMultiPartUpload upload = uploadFactory.recoverRecoverableUpload(s3recoverable); @@ -97,15 +101,27 @@ public S3RecoverableFsDataOutputStream recover(RecoverableWriter.ResumeRecoverab s3recoverable.numBytesInParts()); } + @Override + public boolean requiresCleanupOfRecoverableState() { + return true; + } + + @Override + public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { + final S3Recoverable s3recoverable = castToS3Recoverable(resumable); + final String smallPartObjectToDelete = s3recoverable.incompleteObjectName(); + return smallPartObjectToDelete != null && s3AccessHelper.deleteObject(smallPartObjectToDelete); + } + @Override @SuppressWarnings({"rawtypes", "unchecked"}) - public SimpleVersionedSerializer getCommitRecoverableSerializer() { + public SimpleVersionedSerializer getCommitRecoverableSerializer() { return (SimpleVersionedSerializer) S3RecoverableSerializer.INSTANCE; } @Override @SuppressWarnings({"rawtypes", "unchecked"}) - public SimpleVersionedSerializer getResumeRecoverableSerializer() { + public SimpleVersionedSerializer getResumeRecoverableSerializer() { return (SimpleVersionedSerializer) S3RecoverableSerializer.INSTANCE; } @@ -116,7 +132,7 @@ public boolean supportsResume() { // --------------------------- Utils --------------------------- - private static S3Recoverable castToS3Recoverable(RecoverableWriter.CommitRecoverable recoverable) { + private static S3Recoverable castToS3Recoverable(CommitRecoverable recoverable) { if (recoverable instanceof S3Recoverable) { return (S3Recoverable) recoverable; } @@ -144,6 +160,6 @@ public static S3RecoverableWriter writer( uploadThreadPool, tempFileCreator); - return new S3RecoverableWriter(uploadFactory, tempFileCreator, userDefinedMinPartSize); + return new S3RecoverableWriter(s3AccessHelper, uploadFactory, tempFileCreator, userDefinedMinPartSize); } } diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java index a986111125c22..673796d6c80cb 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java @@ -372,6 +372,11 @@ public PutObjectResult putObject(String key, InputStream file, long length) thro return storeAndGetPutObjectResult(key, content); } + @Override + public boolean deleteObject(String key) throws IOException { + return false; + } + @Override public long getObject(String key, File targetLocation) throws IOException { return 0; diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java index 473439c05530e..b9612adf63a66 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java @@ -85,6 +85,11 @@ public CompleteMultipartUploadResult commitMultiPartUpload(String destKey, Strin return s3accessHelper.completeMPUwithRetries(destKey, uploadId, partETags, length, errorCount); } + @Override + public boolean deleteObject(String key) throws IOException { + return s3a.delete(new org.apache.hadoop.fs.Path('/' + key), false); + } + @Override public long getObject(String key, File targetLocation) throws IOException { long numBytes = 0L; @@ -96,12 +101,20 @@ public long getObject(String key, File targetLocation) throws IOException { final byte[] buffer = new byte[32 * 1024]; int numRead; - while ((numRead = inStream.read(buffer)) > 0) { + while ((numRead = inStream.read(buffer)) != -1) { outStream.write(buffer, 0, numRead); numBytes += numRead; } } + // some sanity checks + if (numBytes != targetLocation.length()) { + throw new IOException(String.format("Error recovering writer: " + + "Downloading the last data chunk file gives incorrect length. " + + "File=%d bytes, Stream=%d bytes", + targetLocation.length(), numBytes)); + } + return numBytes; } diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java index 17fb02b188cc0..6c8619de0ed91 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3RecoverableWriterITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.fs.s3.common.FlinkS3FileSystem; +import org.apache.flink.fs.s3.common.writer.S3Recoverable; import org.apache.flink.testutils.s3.S3TestCredentials; import org.apache.flink.util.MathUtils; import org.apache.flink.util.StringUtils; @@ -42,6 +43,7 @@ import org.junit.rules.TemporaryFolder; import java.io.BufferedReader; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; @@ -213,6 +215,50 @@ public void testCommitAfterPersist() throws Exception { Assert.assertEquals(testData1 + testData2, getContentsOfFile(path)); } + @Test(expected = FileNotFoundException.class) + public void testCleanupRecoverableState() throws Exception { + final RecoverableWriter writer = getRecoverableWriter(); + final Path path = new Path(basePathForTest, "part-0"); + + final RecoverableFsDataOutputStream stream = writer.open(path); + stream.write(bytesOf(testData1)); + S3Recoverable recoverable = (S3Recoverable) stream.persist(); + + stream.closeForCommit().commit(); + + // still the data is there as we have not deleted them from the tmp object + final String content = getContentsOfFile(new Path('/' + recoverable.incompleteObjectName())); + Assert.assertEquals(testData1, content); + + boolean successfullyDeletedState = writer.cleanupRecoverableState(recoverable); + Assert.assertTrue(successfullyDeletedState); + + // this should throw the exception as we deleted the file. + getContentsOfFile(new Path('/' + recoverable.incompleteObjectName())); + } + + @Test + public void testCallingDeleteObjectTwiceDoesNotThroughException() throws Exception { + final RecoverableWriter writer = getRecoverableWriter(); + final Path path = new Path(basePathForTest, "part-0"); + + final RecoverableFsDataOutputStream stream = writer.open(path); + stream.write(bytesOf(testData1)); + S3Recoverable recoverable = (S3Recoverable) stream.persist(); + + stream.closeForCommit().commit(); + + // still the data is there as we have not deleted them from the tmp object + final String content = getContentsOfFile(new Path('/' + recoverable.incompleteObjectName())); + Assert.assertEquals(testData1, content); + + boolean successfullyDeletedState = writer.cleanupRecoverableState(recoverable); + Assert.assertTrue(successfullyDeletedState); + + boolean unsuccessfulDeletion = writer.cleanupRecoverableState(recoverable); + Assert.assertFalse(unsuccessfulDeletion); + } + // ----------------------- Test Recovery ----------------------- @Test diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index 8ba35b8be49db..b59c84ea92e50 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -23,6 +23,8 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.RecoverableFsDataOutputStream; import org.apache.flink.core.fs.RecoverableWriter; +import org.apache.flink.core.fs.RecoverableWriter.CommitRecoverable; +import org.apache.flink.core.fs.RecoverableWriter.ResumeRecoverable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,11 +33,12 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NavigableMap; import java.util.Objects; +import java.util.TreeMap; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -65,14 +68,16 @@ public class Bucket { private final RollingPolicy rollingPolicy; - private final Map> pendingPartsPerCheckpoint; + private final NavigableMap resumablesPerCheckpoint; + + private final NavigableMap> pendingPartsPerCheckpoint; private long partCounter; @Nullable private PartFileWriter inProgressPart; - private List pendingPartsForCurrentCheckpoint; + private List pendingPartsForCurrentCheckpoint; /** * Constructor to create a new empty bucket. @@ -95,7 +100,8 @@ private Bucket( this.rollingPolicy = checkNotNull(rollingPolicy); this.pendingPartsForCurrentCheckpoint = new ArrayList<>(); - this.pendingPartsPerCheckpoint = new HashMap<>(); + this.pendingPartsPerCheckpoint = new TreeMap<>(); + this.resumablesPerCheckpoint = new TreeMap<>(); } /** @@ -123,21 +129,26 @@ private Bucket( } private void restoreInProgressFile(final BucketState state) throws IOException { + if (!state.hasInProgressResumableFile()) { + return; + } // we try to resume the previous in-progress file - if (state.hasInProgressResumableFile()) { - final RecoverableWriter.ResumeRecoverable resumable = state.getInProgressResumableFile(); - final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable); - inProgressPart = partFileFactory.resumeFrom( - bucketId, stream, resumable, state.getInProgressFileCreationTime()); + final ResumeRecoverable resumable = state.getInProgressResumableFile(); + final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable); + inProgressPart = partFileFactory.resumeFrom( + bucketId, stream, resumable, state.getInProgressFileCreationTime()); + + if (fsWriter.requiresCleanupOfRecoverableState()) { + fsWriter.cleanupRecoverableState(resumable); } } private void commitRecoveredPendingFiles(final BucketState state) throws IOException { // we commit pending files for checkpoints that precess the last successful one, from which we are recovering - for (List committables: state.getCommittableFilesPerCheckpoint().values()) { - for (RecoverableWriter.CommitRecoverable committable: committables) { + for (List committables: state.getCommittableFilesPerCheckpoint().values()) { + for (CommitRecoverable committable: committables) { fsWriter.recoverForCommit(committable).commitAfterRecovery(); } } @@ -172,7 +183,7 @@ void merge(final Bucket bucket) throws IOException { checkState(bucket.pendingPartsForCurrentCheckpoint.isEmpty()); checkState(bucket.pendingPartsPerCheckpoint.isEmpty()); - RecoverableWriter.CommitRecoverable committable = bucket.closePartFile(); + CommitRecoverable committable = bucket.closePartFile(); if (committable != null) { pendingPartsForCurrentCheckpoint.add(committable); } @@ -214,8 +225,8 @@ private Path assembleNewPartPath() { return new Path(bucketPath, PART_PREFIX + '-' + subtaskIndex + '-' + partCounter); } - private RecoverableWriter.CommitRecoverable closePartFile() throws IOException { - RecoverableWriter.CommitRecoverable committable = null; + private CommitRecoverable closePartFile() throws IOException { + CommitRecoverable committable = null; if (inProgressPart != null) { committable = inProgressPart.closeForCommit(); pendingPartsForCurrentCheckpoint.add(committable); @@ -233,12 +244,21 @@ void disposePartFile() { BucketState onReceptionOfCheckpoint(long checkpointId) throws IOException { prepareBucketForCheckpointing(checkpointId); - RecoverableWriter.ResumeRecoverable inProgressResumable = null; + ResumeRecoverable inProgressResumable = null; long inProgressFileCreationTime = Long.MAX_VALUE; if (inProgressPart != null) { inProgressResumable = inProgressPart.persist(); inProgressFileCreationTime = inProgressPart.getCreationTime(); + + // the following is an optimization so that writers that do not + // require cleanup, they do not have to keep track of resumables + // and later iterate over the active buckets. + // (see onSuccessfulCompletionOfCheckpoint()) + + if (fsWriter.requiresCleanupOfRecoverableState()) { + this.resumablesPerCheckpoint.put(checkpointId, inProgressResumable); + } } return new BucketState<>(bucketId, bucketPath, inProgressFileCreationTime, inProgressResumable, pendingPartsPerCheckpoint); @@ -261,17 +281,34 @@ private void prepareBucketForCheckpointing(long checkpointId) throws IOException void onSuccessfulCompletionOfCheckpoint(long checkpointId) throws IOException { checkNotNull(fsWriter); - Iterator>> it = - pendingPartsPerCheckpoint.entrySet().iterator(); + Iterator>> it = + pendingPartsPerCheckpoint.headMap(checkpointId, true) + .entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry> entry = it.next(); + + for (CommitRecoverable committable : entry.getValue()) { + fsWriter.recoverForCommit(committable).commit(); + } + it.remove(); + } + + cleanupOutdatedResumables(checkpointId); + } + + private void cleanupOutdatedResumables(long checkpointId) throws IOException { + Iterator> it = + resumablesPerCheckpoint.headMap(checkpointId, false) + .entrySet().iterator(); while (it.hasNext()) { - Map.Entry> entry = it.next(); + final ResumeRecoverable recoverable = it.next().getValue(); + final boolean successfullyDeleted = fsWriter.cleanupRecoverableState(recoverable); + it.remove(); - if (entry.getKey() <= checkpointId) { - for (RecoverableWriter.CommitRecoverable committable : entry.getValue()) { - fsWriter.recoverForCommit(committable).commit(); - } - it.remove(); + if (LOG.isDebugEnabled() && successfullyDeleted) { + LOG.debug("Subtask {} successfully deleted incomplete part for bucket id={}.", subtaskIndex, bucketId); } } } @@ -290,7 +327,7 @@ void onProcessingTime(long timestamp) throws IOException { // --------------------------- Testing Methods ----------------------------- @VisibleForTesting - Map> getPendingPartsPerCheckpoint() { + Map> getPendingPartsPerCheckpoint() { return pendingPartsPerCheckpoint; } @@ -301,7 +338,7 @@ PartFileWriter getInProgressPart() { } @VisibleForTesting - List getPendingPartsForCurrentCheckpoint() { + List getPendingPartsForCurrentCheckpoint() { return pendingPartsForCurrentCheckpoint; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java new file mode 100644 index 0000000000000..f328fd75d25d5 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -0,0 +1,263 @@ +/* + * 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.api.functions.sink.filesystem; + +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableWriter; +import org.apache.flink.core.fs.local.LocalFileSystem; +import org.apache.flink.core.fs.local.LocalRecoverableWriter; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; + +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link Bucket}. + */ +public class BucketTest { + + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Test + public void shouldNotCleanupResumablesThatArePartOfTheAckedCheckpoint() throws IOException { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0); + + bucketUnderTest.write("test-element", 0L); + + final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state, hasActiveInProgressFile()); + + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); + assertThat(recoverableWriter, hasCalledDiscard(0)); // it did not discard as this is still valid. + } + + @Test + public void shouldCleanupOutdatedResumablesOnCheckpointAck() throws IOException { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0); + + bucketUnderTest.write("test-element", 0L); + + final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state, hasActiveInProgressFile()); + + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); + + bucketUnderTest.onReceptionOfCheckpoint(1L); + bucketUnderTest.onReceptionOfCheckpoint(2L); + + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(2L); + assertThat(recoverableWriter, hasCalledDiscard(2)); // that is for checkpoints 0 and 1 + } + + @Test + public void shouldCleanupResumableAfterRestoring() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0); + + bucketUnderTest.write("test-element", 0L); + + final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state, hasActiveInProgressFile()); + + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); + + final TestRecoverableWriter newRecoverableWriter = getRecoverableWriter(path); + restoreBucket(newRecoverableWriter, 0, 1, state); + + assertThat(newRecoverableWriter, hasCalledDiscard(1)); // that is for checkpoints 0 and 1 + } + + @Test + public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0); + + final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state, hasNoActiveInProgressFile()); + + bucketUnderTest.onReceptionOfCheckpoint(1L); + bucketUnderTest.onReceptionOfCheckpoint(2L); + + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(2L); + assertThat(recoverableWriter, hasCalledDiscard(0)); // we have no in-progress file. + } + + // ------------------------------- Matchers -------------------------------- + + private static TypeSafeMatcher hasCalledDiscard(int times) { + return new TypeSafeMatcher() { + @Override + protected boolean matchesSafely(TestRecoverableWriter writer) { + return writer.getCleanupCallCounter() == times; + } + + @Override + public void describeTo(Description description) { + description + .appendText("the TestRecoverableWriter to have called discardRecoverableState() ") + .appendValue(times) + .appendText(" times."); + } + }; + } + + private static TypeSafeMatcher> hasActiveInProgressFile() { + return new TypeSafeMatcher>() { + @Override + protected boolean matchesSafely(BucketState state) { + return state.getInProgressResumableFile() != null; + } + + @Override + public void describeTo(Description description) { + description.appendText("a BucketState with active in-progress file."); + } + }; + } + + private static TypeSafeMatcher> hasNoActiveInProgressFile() { + return new TypeSafeMatcher>() { + @Override + protected boolean matchesSafely(BucketState state) { + return state.getInProgressResumableFile() == null; + } + + @Override + public void describeTo(Description description) { + description.appendText("a BucketState with no active in-progress file."); + } + }; + } + + // ------------------------------- Mock Classes -------------------------------- + + private static class TestRecoverableWriter extends LocalRecoverableWriter { + + private int cleanupCallCounter = 0; + + TestRecoverableWriter(LocalFileSystem fs) { + super(fs); + } + + int getCleanupCallCounter() { + return cleanupCallCounter; + } + + @Override + public boolean requiresCleanupOfRecoverableState() { + // here we return true so that the cleanupRecoverableState() is called. + return true; + } + + @Override + public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { + cleanupCallCounter++; + return false; + } + + @Override + public String toString() { + return "TestRecoverableWriter has called discardRecoverableState() " + cleanupCallCounter + " times."; + } + } + + // ------------------------------- Utility Methods -------------------------------- + + private static final String bucketId = "testing-bucket"; + + private static final RollingPolicy rollingPolicy = DefaultRollingPolicy.create().build(); + + private static final PartFileWriter.PartFileFactory partFileFactory = + new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()); + + private static Bucket createBucket( + final RecoverableWriter writer, + final Path bucketPath, + final int subtaskIdx, + final int initialPartCounter) { + + return Bucket.getNew( + writer, + subtaskIdx, + bucketId, + bucketPath, + initialPartCounter, + partFileFactory, + rollingPolicy); + } + + private static Bucket restoreBucket( + final RecoverableWriter writer, + final int subtaskIndex, + final long initialPartCounter, + final BucketState bucketState) throws Exception { + + return Bucket.restore( + writer, + subtaskIndex, + initialPartCounter, + partFileFactory, + rollingPolicy, + bucketState + ); + } + + private static TestRecoverableWriter getRecoverableWriter(Path path) { + try { + final FileSystem fs = FileSystem.get(path.toUri()); + if (!(fs instanceof LocalFileSystem)) { + fail("Expected Local FS but got a " + fs.getClass().getName() + " for path: " + path); + } + return new TestRecoverableWriter((LocalFileSystem) fs); + } catch (IOException e) { + fail(); + } + return null; + } +} From 652302863a5f1b5f07b9cd80692d93bf9e449181 Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Sun, 2 Dec 2018 19:05:49 +0800 Subject: [PATCH 192/359] [FLINK-11045][table] Set correct UserCodeClassLoader for RuntimeUDFContext in CollectionExecutor This closes #7213 --- .../common/operators/CollectionExecutor.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) 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 55f3df7d31c0e..3a0b84c844987 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 @@ -83,7 +83,7 @@ public class CollectionExecutor { private final Map> aggregators; - private final ClassLoader classLoader; + private final ClassLoader userCodeClassLoader; private final ExecutionConfig executionConfig; @@ -99,7 +99,7 @@ public CollectionExecutor(ExecutionConfig executionConfig) { this.previousAggregates = new HashMap(); this.aggregators = new HashMap>(); this.cachedFiles = new HashMap>(); - this.classLoader = getClass().getClassLoader(); + this.userCodeClassLoader = Thread.currentThread().getContextClassLoader(); } // -------------------------------------------------------------------------------------------- @@ -191,8 +191,8 @@ private void executeDataSink(GenericDataSinkBase sink, int superStep) th MetricGroup metrics = new UnregisteredMetricsGroup(); if (RichOutputFormat.class.isAssignableFrom(typedSink.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics) : - new IterationRuntimeUDFContext(taskInfo, classLoader, executionConfig, cachedFiles, accumulators, metrics); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, userCodeClassLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, userCodeClassLoader, executionConfig, cachedFiles, accumulators, metrics); } else { ctx = null; } @@ -211,8 +211,8 @@ private List executeDataSource(GenericDataSourceBase source, in MetricGroup metrics = new UnregisteredMetricsGroup(); 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); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, userCodeClassLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, userCodeClassLoader, executionConfig, cachedFiles, accumulators, metrics); } else { ctx = null; } @@ -237,8 +237,8 @@ private List executeUnaryOperator(SingleInputOperator op MetricGroup metrics = new UnregisteredMetricsGroup(); 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); + ctx = superStep == 0 ? new RuntimeUDFContext(taskInfo, userCodeClassLoader, executionConfig, cachedFiles, accumulators, metrics) : + new IterationRuntimeUDFContext(taskInfo, userCodeClassLoader, executionConfig, cachedFiles, accumulators, metrics); for (Map.Entry> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -278,8 +278,8 @@ private List executeBinaryOperator(DualInputOperator> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); From a15a29837b9e100f21ad8e8d79821ef6b0490871 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 4 Dec 2018 17:35:42 +0100 Subject: [PATCH 193/359] [FLINK-10522][fs-connector] Check if RecoverableWriter supportsResume() and act accordingly. --- .../api/functions/sink/filesystem/Bucket.java | 14 +- .../functions/sink/filesystem/BucketTest.java | 185 +++++++++++++++++- .../sink/filesystem/utils/NoOpCommitter.java | 49 +++++ .../filesystem/utils/NoOpRecoverable.java | 32 +++ .../NoOpRecoverableFsDataOutputStream.java | 67 +++++++ .../utils/NoOpRecoverableWriter.java | 75 +++++++ 6 files changed, 418 insertions(+), 4 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpCommitter.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverable.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableFsDataOutputStream.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableWriter.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index b59c84ea92e50..3252d9c0225bd 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -135,9 +135,17 @@ private void restoreInProgressFile(final BucketState state) throws IOE // we try to resume the previous in-progress file final ResumeRecoverable resumable = state.getInProgressResumableFile(); - final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable); - inProgressPart = partFileFactory.resumeFrom( - bucketId, stream, resumable, state.getInProgressFileCreationTime()); + + if (fsWriter.supportsResume()) { + final RecoverableFsDataOutputStream stream = fsWriter.recover(resumable); + inProgressPart = partFileFactory.resumeFrom( + bucketId, stream, resumable, state.getInProgressFileCreationTime()); + } else { + // if the writer does not support resume, then we close the + // in-progress part and commit it, as done in the case of pending files. + + fsWriter.recoverForCommit(resumable).commitAfterRecovery(); + } if (fsWriter.requiresCleanupOfRecoverableState()) { fsWriter.cleanupRecoverableState(resumable); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java index f328fd75d25d5..308bc31f1848e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -21,25 +21,36 @@ import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; import org.apache.flink.core.fs.RecoverableWriter; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.fs.local.LocalRecoverableWriter; import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; +import org.apache.flink.streaming.api.functions.sink.filesystem.utils.NoOpCommitter; +import org.apache.flink.streaming.api.functions.sink.filesystem.utils.NoOpRecoverable; +import org.apache.flink.streaming.api.functions.sink.filesystem.utils.NoOpRecoverableFsDataOutputStream; +import org.apache.flink.streaming.api.functions.sink.filesystem.utils.NoOpRecoverableWriter; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; +import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; /** - * Tests for the {@link Bucket}. + * Tests for the {@code Bucket}. */ public class BucketTest { @@ -128,6 +139,37 @@ public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { assertThat(recoverableWriter, hasCalledDiscard(0)); // we have no in-progress file. } + // --------------------------- Checking Restore --------------------------- + + @Test + public void inProgressFileShouldBeCommittedIfWriterDoesNotSupportResume() throws IOException { + final StubNonResumableWriter nonResumableWriter = new StubNonResumableWriter(); + final Bucket bucket = getRestoredBucketWithOnlyInProgressPart(nonResumableWriter); + + Assert.assertThat(nonResumableWriter, hasMethodCallCountersEqualTo(1, 0, 1)); + Assert.assertThat(bucket, hasNullInProgressFile(true)); + } + + @Test + public void inProgressFileShouldBeRestoredIfWriterSupportsResume() throws IOException { + final StubResumableWriter resumableWriter = new StubResumableWriter(); + final Bucket bucket = getRestoredBucketWithOnlyInProgressPart(resumableWriter); + + Assert.assertThat(resumableWriter, hasMethodCallCountersEqualTo(1, 1, 0)); + Assert.assertThat(bucket, hasNullInProgressFile(false)); + } + + @Test + public void pendingFilesShouldBeRestored() throws IOException { + final int expectedRecoverForCommitCounter = 10; + + final StubNonResumableWriter writer = new StubNonResumableWriter(); + final Bucket bucket = getRestoredBucketWithOnlyPendingParts(writer, expectedRecoverForCommitCounter); + + Assert.assertThat(writer, hasMethodCallCountersEqualTo(0, 0, expectedRecoverForCommitCounter)); + Assert.assertThat(bucket, hasNullInProgressFile(true)); + } + // ------------------------------- Matchers -------------------------------- private static TypeSafeMatcher hasCalledDiscard(int times) { @@ -175,6 +217,47 @@ public void describeTo(Description description) { }; } + private static TypeSafeMatcher> hasNullInProgressFile(final boolean isNull) { + + return new TypeSafeMatcher>() { + @Override + protected boolean matchesSafely(Bucket bucket) { + final PartFileWriter inProgressPart = bucket.getInProgressPart(); + return isNull == (inProgressPart == null); + } + + @Override + public void describeTo(Description description) { + description.appendText("a Bucket with its inProgressPart being ") + .appendText(isNull ? " null." : " not null."); + } + }; + } + + private static TypeSafeMatcher hasMethodCallCountersEqualTo( + final int supportsResumeCalls, + final int recoverCalls, + final int recoverForCommitCalls) { + + return new TypeSafeMatcher() { + @Override + protected boolean matchesSafely(BaseStubWriter writer) { + return writer.getSupportsResumeCallCounter() == supportsResumeCalls && + writer.getRecoverCallCounter() == recoverCalls && + writer.getRecoverForCommitCallCounter() == recoverForCommitCalls; + } + + @Override + public void describeTo(Description description) { + description.appendText("a Writer where:") + .appendText(" supportsResume was called ").appendValue(supportsResumeCalls).appendText(" times,") + .appendText(" recover was called ").appendValue(recoverCalls).appendText(" times,") + .appendText(" and recoverForCommit was called ").appendValue(recoverForCommitCalls).appendText(" times.") + .appendText("'"); + } + }; + } + // ------------------------------- Mock Classes -------------------------------- private static class TestRecoverableWriter extends LocalRecoverableWriter { @@ -207,6 +290,81 @@ public String toString() { } } + /** + * A test implementation of a {@link RecoverableWriter} that does not support + * resuming, i.e. keep on writing to the in-progress file at the point we were + * before the failure. + */ + private static class StubResumableWriter extends BaseStubWriter { + + StubResumableWriter() { + super(true); + } + } + + /** + * A test implementation of a {@link RecoverableWriter} that does not support + * resuming, i.e. keep on writing to the in-progress file at the point we were + * before the failure. + */ + private static class StubNonResumableWriter extends BaseStubWriter { + + StubNonResumableWriter() { + super(false); + } + } + + /** + * A test implementation of a {@link RecoverableWriter} that does not support + * resuming, i.e. keep on writing to the in-progress file at the point we were + * before the failure. + */ + private static class BaseStubWriter extends NoOpRecoverableWriter { + + private final boolean supportsResume; + + private int supportsResumeCallCounter = 0; + + private int recoverCallCounter = 0; + + private int recoverForCommitCallCounter = 0; + + private BaseStubWriter(final boolean supportsResume) { + this.supportsResume = supportsResume; + } + + int getSupportsResumeCallCounter() { + return supportsResumeCallCounter; + } + + int getRecoverCallCounter() { + return recoverCallCounter; + } + + int getRecoverForCommitCallCounter() { + return recoverForCommitCallCounter; + } + + @Override + public RecoverableFsDataOutputStream recover(ResumeRecoverable resumable) throws IOException { + recoverCallCounter++; + return new NoOpRecoverableFsDataOutputStream(); + } + + @Override + public RecoverableFsDataOutputStream.Committer recoverForCommit(CommitRecoverable resumable) throws IOException { + checkArgument(resumable instanceof NoOpRecoverable); + recoverForCommitCallCounter++; + return new NoOpCommitter(); + } + + @Override + public boolean supportsResume() { + supportsResumeCallCounter++; + return supportsResume; + } + } + // ------------------------------- Utility Methods -------------------------------- private static final String bucketId = "testing-bucket"; @@ -260,4 +418,29 @@ private static TestRecoverableWriter getRecoverableWriter(Path path) { } return null; } + + private Bucket getRestoredBucketWithOnlyInProgressPart(final BaseStubWriter writer) throws IOException { + final BucketState stateWithOnlyInProgressFile = + new BucketState<>("test", new Path(), 12345L, new NoOpRecoverable(), new HashMap<>()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile); + } + + private Bucket getRestoredBucketWithOnlyPendingParts(final BaseStubWriter writer, final int numberOfPendingParts) throws IOException { + final Map> completePartsPerCheckpoint = + createPendingPartsPerCheckpoint(numberOfPendingParts); + + final BucketState initStateWithOnlyInProgressFile = + new BucketState<>("test", new Path(), 12345L, null, completePartsPerCheckpoint); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile); + } + + private Map> createPendingPartsPerCheckpoint(int noOfCheckpoints) { + final Map> pendingCommittablesPerCheckpoint = new HashMap<>(); + for (int checkpointId = 0; checkpointId < noOfCheckpoints; checkpointId++) { + final List pending = new ArrayList<>(); + pending.add(new NoOpRecoverable()); + pendingCommittablesPerCheckpoint.put((long) checkpointId, pending); + } + return pendingCommittablesPerCheckpoint; + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpCommitter.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpCommitter.java new file mode 100644 index 0000000000000..06005a15a4869 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpCommitter.java @@ -0,0 +1,49 @@ +/* + * 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.api.functions.sink.filesystem.utils; + +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; +import org.apache.flink.core.fs.RecoverableWriter; + +import java.io.IOException; + +/** + * An implementation of the {@link RecoverableFsDataOutputStream.Committer committer} + * that does nothing. + * + *

    This is to avoid to have to implement all methods for every implementation + * used in tests. + */ +public class NoOpCommitter implements RecoverableFsDataOutputStream.Committer { + + @Override + public void commit() throws IOException { + + } + + @Override + public void commitAfterRecovery() throws IOException { + + } + + @Override + public RecoverableWriter.CommitRecoverable getRecoverable() { + return null; + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverable.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverable.java new file mode 100644 index 0000000000000..e00d4c4b0a9e0 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverable.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.streaming.api.functions.sink.filesystem.utils; + +import org.apache.flink.core.fs.RecoverableWriter; + +/** + * An implementation of the {@link RecoverableWriter.ResumeRecoverable ResumeRecoverable} + * that does nothing. + * + *

    This is to avoid to have to implement all methods for every implementation + * used in tests. + */ +public class NoOpRecoverable implements RecoverableWriter.ResumeRecoverable { + +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableFsDataOutputStream.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableFsDataOutputStream.java new file mode 100644 index 0000000000000..a549896922daf --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableFsDataOutputStream.java @@ -0,0 +1,67 @@ +/* + * 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.api.functions.sink.filesystem.utils; + +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; +import org.apache.flink.core.fs.RecoverableWriter; + +import java.io.IOException; + +/** + * A default implementation of the {@link RecoverableFsDataOutputStream} that does nothing. + * + *

    This is to avoid to have to implement all methods for every implementation + * used in tests. + */ +public class NoOpRecoverableFsDataOutputStream extends RecoverableFsDataOutputStream { + @Override + public RecoverableWriter.ResumeRecoverable persist() throws IOException { + return null; + } + + @Override + public Committer closeForCommit() throws IOException { + return null; + } + + @Override + public void close() throws IOException { + + } + + @Override + public long getPos() throws IOException { + return 0; + } + + @Override + public void flush() throws IOException { + + } + + @Override + public void sync() throws IOException { + + } + + @Override + public void write(int b) throws IOException { + + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableWriter.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableWriter.java new file mode 100644 index 0000000000000..e21da2aeabd73 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/utils/NoOpRecoverableWriter.java @@ -0,0 +1,75 @@ +/* + * 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.api.functions.sink.filesystem.utils; + +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.RecoverableFsDataOutputStream; +import org.apache.flink.core.fs.RecoverableWriter; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** + * A default implementation of the {@link RecoverableWriter} that does nothing. + * + *

    This is to avoid to have to implement all methods for every implementation + * used in tests. + */ +public class NoOpRecoverableWriter implements RecoverableWriter { + + @Override + public RecoverableFsDataOutputStream open(Path path) throws IOException { + return null; + } + + @Override + public RecoverableFsDataOutputStream recover(RecoverableWriter.ResumeRecoverable resumable) throws IOException { + return null; + } + + @Override + public boolean requiresCleanupOfRecoverableState() { + return false; + } + + @Override + public boolean cleanupRecoverableState(ResumeRecoverable resumable) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public RecoverableFsDataOutputStream.Committer recoverForCommit(RecoverableWriter.CommitRecoverable resumable) throws IOException { + return null; + } + + @Override + public SimpleVersionedSerializer getCommitRecoverableSerializer() { + return null; + } + + @Override + public SimpleVersionedSerializer getResumeRecoverableSerializer() { + return null; + } + + @Override + public boolean supportsResume() { + return false; + } +} From 0215f498382c3dc244628666e4a24c5b837492fa Mon Sep 17 00:00:00 2001 From: Igal Shilman Date: Tue, 4 Dec 2018 16:45:54 +0100 Subject: [PATCH 194/359] [hotfix] Fixing the broken code examples The code examples for Scala and Java are both broken, and set a bad example in terms of efficiency. This closes #7232. --- docs/dev/connectors/streamfile_sink.md | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index 8f50675ccbc59..82ab5620571ba 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -60,17 +60,14 @@ Basic usage thus looks like this:

    {% highlight java %} -import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; DataStream input = ...; final StreamingFileSink sink = StreamingFileSink - .forRowFormat(new Path(outputPath), (Encoder) (element, stream) -> { - PrintStream out = new PrintStream(stream); - out.println(element.f1); - }) + .forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8")) .build(); input.addSink(sink); @@ -79,19 +76,16 @@ input.addSink(sink);
    {% highlight scala %} -import org.apache.flink.api.common.serialization.Encoder +import org.apache.flink.api.common.serialization.SimpleStringEncoder import org.apache.flink.core.fs.Path import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink val input: DataStream[String] = ... -final StreamingFileSink[String] sink = StreamingFileSink - .forRowFormat(new Path(outputPath), (element, stream) => { - val out = new PrintStream(stream) - out.println(element.f1) - }) - .build() - +val sink: StreamingFileSink[String] = StreamingFileSink + .forRowFormat(new Path(outputPath), new SimpleStringEncoder[String]("UTF-8")) + .build() + input.addSink(sink) {% endhighlight %} From efc73a872ac52e314bb1a05b9c5ed045cde6df1f Mon Sep 17 00:00:00 2001 From: chensq Date: Fri, 16 Nov 2018 15:33:23 +0800 Subject: [PATCH 195/359] [FLINK-9555][scala-shell] Support table api in scala shell. This closes #7121 --- flink-scala-shell/pom.xml | 7 +++ .../apache/flink/api/scala/FlinkILoop.scala | 39 ++++++++++--- .../flink/api/scala/ScalaShellITCase.scala | 55 +++++++++++++++++++ .../start-script/start-scala-shell.sh | 7 +++ 4 files changed, 99 insertions(+), 9 deletions(-) diff --git a/flink-scala-shell/pom.xml b/flink-scala-shell/pom.xml index 5a96804794590..dfb7416b36104 100644 --- a/flink-scala-shell/pom.xml +++ b/flink-scala-shell/pom.xml @@ -78,6 +78,13 @@ under the License. ${scala.version} + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala index 4b6e886994abc..c124d8ea8a3c7 100644 --- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala +++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala @@ -23,6 +23,8 @@ import java.io.{BufferedReader, File, FileOutputStream} import org.apache.flink.api.java.{JarHelper, ScalaShellRemoteEnvironment, ScalaShellRemoteStreamEnvironment} import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.configuration.Configuration +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.scala.{BatchTableEnvironment, StreamTableEnvironment} import org.apache.flink.util.AbstractID import scala.tools.nsc.interpreter._ @@ -90,10 +92,17 @@ class FlinkILoop( } // local environment - val (scalaBenv: ExecutionEnvironment, scalaSenv: StreamExecutionEnvironment) = { + val ( + scalaBenv: ExecutionEnvironment, + scalaSenv: StreamExecutionEnvironment, + scalaBTEnv: BatchTableEnvironment, + scalaSTEnv: StreamTableEnvironment + ) = { val scalaBenv = new ExecutionEnvironment(remoteBenv) val scalaSenv = new StreamExecutionEnvironment(remoteSenv) - (scalaBenv,scalaSenv) + val scalaBTEnv = TableEnvironment.getTableEnvironment(scalaBenv) + val scalaSTEnv = TableEnvironment.getTableEnvironment(scalaSenv) + (scalaBenv,scalaSenv,scalaBTEnv,scalaSTEnv) } /** @@ -139,7 +148,10 @@ class FlinkILoop( "org.apache.flink.api.scala._", "org.apache.flink.api.scala.utils._", "org.apache.flink.streaming.api.scala._", - "org.apache.flink.streaming.api.windowing.time._" + "org.apache.flink.streaming.api.windowing.time._", + "org.apache.flink.table.api._", + "org.apache.flink.table.api.scala._", + "org.apache.flink.types.Row" ) override def createInterpreter(): Unit = { @@ -152,6 +164,8 @@ class FlinkILoop( // set execution environment intp.bind("benv", this.scalaBenv) intp.bind("senv", this.scalaSenv) + intp.bind("btenv", this.scalaBTEnv) + intp.bind("stenv", this.scalaSTEnv) } } @@ -243,22 +257,29 @@ class FlinkILoop( F L I N K - S C A L A - S H E L L -NOTE: Use the prebound Execution Environments to implement batch or streaming programs. +NOTE: Use the prebound Execution Environments and Table Environment to implement batch or streaming programs. - Batch - Use the 'benv' variable + Batch - Use the 'benv' and 'btenv' variable * val dataSet = benv.readTextFile("/path/to/data") * dataSet.writeAsText("/path/to/output") * benv.execute("My batch program") + * + * val batchTable = btenv.fromDataSet(dataSet) + * btenv.registerTable("tableName", batchTable) + * val result = btenv.sqlQuery("SELECT * FROM tableName").collect + HINT: You can use print() on a DataSet to print the contents or collect() + a sql query result back to the shell. - HINT: You can use print() on a DataSet to print the contents to the shell. - - Streaming - Use the 'senv' variable + Streaming - Use the 'senv' and 'stenv' variable * val dataStream = senv.fromElements(1, 2, 3, 4) * dataStream.countWindowAll(2).sum(0).print() + * + * val streamTable = stenv.fromDataStream(dataStream, 'num) + * val resultTable = streamTable.select('num).where('num % 2 === 1 ) + * resultTable.toAppendStream[Row].print() * senv.execute("My streaming program") - HINT: You can only print a DataStream to the shell in local mode. """ // scalastyle:on diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala index 337e4fb9be9e6..fc90d8d143c60 100644 --- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala +++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala @@ -168,6 +168,61 @@ class ScalaShellITCase extends TestLogger { Assert.assertTrue(output.contains("WC(world,10)")) } + @Test + def testSimpleSelectWithFilterBatchTableAPIQuery: Unit = { + val input = + """ + |val data = Seq( + | (1, 1L, "Hi"), + | (2, 2L, "Hello"), + | (3, 2L, "Hello world")) + |val t = benv.fromCollection(data).toTable(btenv, 'a, 'b, 'c).select('a,'c).where( + |'a% 2 === 1 ) + |val results = t.toDataSet[Row].collect() + |results.foreach(println) + |:q + """.stripMargin + val output = processInShell(input) + Assert.assertFalse(output.contains("failed")) + Assert.assertFalse(output.contains("error")) + Assert.assertFalse(output.contains("Exception")) + Assert.assertTrue(output.contains("1,Hi")) + Assert.assertTrue(output.contains("3,Hello world")) + } + + @Test + def testGroupedAggregationStreamTableAPIQuery: Unit = { + val input = + """ + | val data = List( + | ("Hello", 1), + | ("word", 1), + | ("Hello", 1), + | ("bark", 1), + | ("bark", 1), + | ("bark", 1), + | ("bark", 1), + | ("bark", 1), + | ("bark", 1), + | ("flink", 1) + | ) + | val stream = senv.fromCollection(data) + | val table = stream.toTable(stenv, 'word, 'num) + | val resultTable = table.groupBy('word).select('num.sum as 'count).groupBy('count).select( + | 'count,'count.count as 'frequency) + | val results = resultTable.toRetractStream[Row] + | results.print + | senv.execute + """.stripMargin + val output = processInShell(input) + Assert.assertTrue(output.contains("6,1")) + Assert.assertTrue(output.contains("1,2")) + Assert.assertTrue(output.contains("2,1")) + Assert.assertFalse(output.contains("failed")) + Assert.assertFalse(output.contains("error")) + Assert.assertFalse(output.contains("Exception")) + } + /** * Submit external library. * Disabled due to FLINK-7111. diff --git a/flink-scala-shell/start-script/start-scala-shell.sh b/flink-scala-shell/start-script/start-scala-shell.sh index 033d505032229..e3571145f13ed 100644 --- a/flink-scala-shell/start-script/start-scala-shell.sh +++ b/flink-scala-shell/start-script/start-scala-shell.sh @@ -52,6 +52,13 @@ bin=`cd "$bin"; pwd` FLINK_CLASSPATH=`constructFlinkClassPath` +# Append flink-table jar into class path +opt=`dirname "$0"` +opt=`cd ../"$opt"/opt; pwd` +FLINK_TABLE_LIB_PATH=$opt/`ls $opt|grep flink-table_*` +FLINK_CLASSPATH=$FLINK_CLASSPATH:$FLINK_TABLE_LIB_PATH + + # https://issues.scala-lang.org/browse/SI-6502, cant load external jars interactively # in scala shell since 2.10, has to be done at startup # checks arguments for additional classpath and adds it to the "standard classpath" From 1a40a08cf44cf01b2bdb055421b18ed0f2ea8829 Mon Sep 17 00:00:00 2001 From: KarmaGYZ Date: Sun, 2 Dec 2018 21:27:27 +0800 Subject: [PATCH 196/359] [hotfix][docs] Fix invalid link in schema_evolution doc --- docs/dev/stream/state/schema_evolution.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/stream/state/schema_evolution.md b/docs/dev/stream/state/schema_evolution.md index 2fb10a74ff1f0..e4c2d4aa44189 100644 --- a/docs/dev/stream/state/schema_evolution.md +++ b/docs/dev/stream/state/schema_evolution.md @@ -74,7 +74,7 @@ serialization schema than the previous serializer; if so, the previous serialize and written back to bytes again with the new serializer. Further details about the migration process is out of the scope of this documentation; please refer to -[here]({{ site.baseurl }}/dev/stream/state/custom_serialization). +[here]({{ site.baseurl }}/dev/stream/state/custom_serialization.html). ## Supported data types for schema evolution From 5b945f73ea47364badb12ca0b9c45604b8028e73 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 23 Nov 2018 15:57:02 +0100 Subject: [PATCH 197/359] [FLINK-10997][formats] Bundle kafka-scheme-registry-client --- .../flink-avro-confluent-registry/pom.xml | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/flink-formats/flink-avro-confluent-registry/pom.xml b/flink-formats/flink-avro-confluent-registry/pom.xml index 9d818519672bd..9b93e3572ac64 100644 --- a/flink-formats/flink-avro-confluent-registry/pom.xml +++ b/flink-formats/flink-avro-confluent-registry/pom.xml @@ -78,10 +78,31 @@ under the License. shade + false + + + io.confluent:* + com.fasterxml.jackson.core:* + org.apache.zookeeper:zookeeper + com.101tec:zkclient + + - com.fasterxml.jackson.core - org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson.core + com.fasterxml.jackson + org.apache.flink.formats.avro.registry.confluent.shaded.com.fasterxml.jackson + + + org.apache.zookeeper + org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.zookeeper + + + org.apache.jute + org.apache.flink.formats.avro.registry.confluent.shaded.org.apache.jute + + + org.I0Itec.zkclient + org.apache.flink.formats.avro.registry.confluent.shaded.org.101tec From a3ae8c0762858e3813c2b042f5915d9e2b948a35 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Nov 2018 16:07:33 +0100 Subject: [PATCH 198/359] [FLINK-10987] Add LICENSE & NOTICE files for flink-avro-confluent-registry --- .../src/main/resources/META-INF/NOTICE | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE diff --git a/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..24f2ff846578c --- /dev/null +++ b/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -0,0 +1,15 @@ +flink-avro-confluent-registry +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.101tec:zkclient:0.10 +- com.fasterxml.jackson.core:jackson-databind:2.8.4 +- com.fasterxml.jackson.core:jackson-annotations:2.8.0 +- com.fasterxml.jackson.core:jackson-core:2.8.4 +- io.confluent:common-utils:3.3.1 +- io.confluent:kafka-schema-registry-client:3.3.1 +- org.apache.zookeeper:zookeeper:3.4.10 From 1c1de7415002fc889510e8c84f807efbce149e5a Mon Sep 17 00:00:00 2001 From: azagrebin Date: Thu, 6 Dec 2018 12:08:40 +0100 Subject: [PATCH 199/359] [FLINK-11011][E2E][JM] Log error messages about null CheckpointCoordinator only if job is running (#7224) --- .../flink/runtime/jobmaster/JobMaster.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 40a675aca3146..0b245e74ff460 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -683,8 +683,12 @@ public void acknowledgeCheckpoint( } }); } else { - log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator", - jobGraph.getJobID()); + String errorMessage = "Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator"; + if (executionGraph.getState() == JobStatus.RUNNING) { + log.error(errorMessage, jobGraph.getJobID()); + } else { + log.debug(errorMessage, jobGraph.getJobID()); + } } } @@ -702,8 +706,12 @@ public void declineCheckpoint(DeclineCheckpoint decline) { } }); } else { - log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator", - jobGraph.getJobID()); + String errorMessage = "Received DeclineCheckpoint message for job {} with no CheckpointCoordinator"; + if (executionGraph.getState() == JobStatus.RUNNING) { + log.error(errorMessage, jobGraph.getJobID()); + } else { + log.debug(errorMessage, jobGraph.getJobID()); + } } } From f19bc72e910615a4d122f2fe3777fde6774bc001 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 15 Nov 2018 18:51:43 +0100 Subject: [PATCH 200/359] [FLINK-10482] Fix double counting of checkpoint stat --- .../checkpoint/CheckpointStatsCounts.java | 24 ++++++++++++----- .../flink/runtime/jobmaster/JobMaster.java | 17 ++++++------ .../checkpoint/CheckpointStatsCountsTest.java | 27 +++++++++---------- 3 files changed, 38 insertions(+), 30 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java index dad45eb669c0f..9e15aebd048fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCounts.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.checkpoint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkArgument; @@ -26,6 +29,7 @@ * Counts of checkpoints. */ public class CheckpointStatsCounts implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(CheckpointStatsCounts.class); private static final long serialVersionUID = -5229425063269482528L; @@ -147,9 +151,8 @@ void incrementInProgressCheckpoints() { * {@link #incrementInProgressCheckpoints()}. */ void incrementCompletedCheckpoints() { - if (--numInProgressCheckpoints < 0) { - throw new IllegalStateException("Incremented the completed number of checkpoints " + - "without incrementing the in progress checkpoints before."); + if (canDecrementOfInProgressCheckpointsNumber()) { + numInProgressCheckpoints--; } numCompletedCheckpoints++; } @@ -161,9 +164,8 @@ void incrementCompletedCheckpoints() { * {@link #incrementInProgressCheckpoints()}. */ void incrementFailedCheckpoints() { - if (--numInProgressCheckpoints < 0) { - throw new IllegalStateException("Incremented the completed number of checkpoints " + - "without incrementing the in progress checkpoints before."); + if (canDecrementOfInProgressCheckpointsNumber()) { + numInProgressCheckpoints--; } numFailedCheckpoints++; } @@ -181,4 +183,14 @@ CheckpointStatsCounts createSnapshot() { numCompletedCheckpoints, numFailedCheckpoints); } + + private boolean canDecrementOfInProgressCheckpointsNumber() { + boolean decrementLeadsToNegativeNumber = numInProgressCheckpoints - 1 < 0; + if (decrementLeadsToNegativeNumber) { + String errorMessage = "Incremented the completed number of checkpoints " + + "without incrementing the in progress checkpoints before."; + LOG.warn(errorMessage); + } + return !decrementLeadsToNegativeNumber; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 0b245e74ff460..2fdf79c7cb2fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -964,19 +964,18 @@ public CompletableFuture triggerSavepoint( return checkpointCoordinator .triggerSavepoint(System.currentTimeMillis(), targetDirectory) .thenApply(CompletedCheckpoint::getExternalPointer) - .thenApplyAsync(path -> { - if (cancelJob) { + .handleAsync((path, throwable) -> { + if (throwable != null) { + if (cancelJob) { + startCheckpointScheduler(checkpointCoordinator); + } + throw new CompletionException(throwable); + } else if (cancelJob) { log.info("Savepoint stored in {}. Now cancelling {}.", path, jobGraph.getJobID()); cancel(timeout); } return path; - }, getMainThreadExecutor()) - .exceptionally(throwable -> { - if (cancelJob) { - startCheckpointScheduler(checkpointCoordinator); - } - throw new CompletionException(throwable); - }); + }, getMainThreadExecutor()); } private void startCheckpointScheduler(final CheckpointCoordinator checkpointCoordinator) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCountsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCountsTest.java index cf1e7f7f82d04..2d09b46464ff7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCountsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsCountsTest.java @@ -21,15 +21,16 @@ import org.junit.Test; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.junit.Assert.assertTrue; +/** Test checkpoint statistics counters. */ public class CheckpointStatsCountsTest { /** * Tests that counts are reported correctly. */ @Test - public void testCounts() throws Exception { + public void testCounts() { CheckpointStatsCounts counts = new CheckpointStatsCounts(); assertEquals(0, counts.getNumberOfRestoredCheckpoints()); assertEquals(0, counts.getTotalNumberOfCheckpoints()); @@ -80,19 +81,15 @@ public void testCounts() throws Exception { * incrementing the in progress checkpoints before throws an Exception. */ @Test - public void testCompleteOrFailWithoutInProgressCheckpoint() throws Exception { + public void testCompleteOrFailWithoutInProgressCheckpoint() { CheckpointStatsCounts counts = new CheckpointStatsCounts(); - try { - counts.incrementCompletedCheckpoints(); - fail("Did not throw expected Exception"); - } catch (IllegalStateException ignored) { - } - - try { - counts.incrementFailedCheckpoints(); - fail("Did not throw expected Exception"); - } catch (IllegalStateException ignored) { - } + counts.incrementCompletedCheckpoints(); + assertTrue("Number of checkpoints in progress should never be negative", + counts.getNumberOfInProgressCheckpoints() >= 0); + + counts.incrementFailedCheckpoints(); + assertTrue("Number of checkpoints in progress should never be negative", + counts.getNumberOfInProgressCheckpoints() >= 0); } /** @@ -100,7 +97,7 @@ public void testCompleteOrFailWithoutInProgressCheckpoint() throws Exception { * parent. */ @Test - public void testCreateSnapshot() throws Exception { + public void testCreateSnapshot() { CheckpointStatsCounts counts = new CheckpointStatsCounts(); counts.incrementRestoredCheckpoints(); counts.incrementRestoredCheckpoints(); From 5f369617649ed813a9560b181c3f278920bb20a6 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Fri, 7 Dec 2018 16:15:58 +0100 Subject: [PATCH 201/359] [hotfix][documentation] Mention limitation of local recovery with RocksDB in multi device setups. --- docs/ops/state/large_state_tuning.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md index 62b3ee557f6d3..50c5a533caef7 100644 --- a/docs/ops/state/large_state_tuning.md +++ b/docs/ops/state/large_state_tuning.md @@ -324,7 +324,9 @@ and occupy local disk space. In the future, we might also offer an implementatio and occupy local disk space. For *incremental snapshots*, the local state is based on RocksDB's native checkpointing mechanism. This mechanism is also used as the first step to create the primary copy, which means that in this case no additional cost is introduced for creating the secondary copy. We simply keep the native checkpoint directory around instead of deleting it after uploading to the distributed store. This local copy can share active files with the working directory of RocksDB (via hard links), so for active -files also no additional disk space is consumed for task-local recovery with incremental snapshots. +files also no additional disk space is consumed for task-local recovery with incremental snapshots. Using hard links also means that the RocksDB directories must be on +the same physical device as all the configure local recovery directories that can be used to store local state, or else establishing hard links can fail (see FLINK-10954). +Currently, this also prevents using local recovery when RocksDB directories are configured to be located on more than one physical device. ### Allocation-preserving scheduling From 0fa9ec030d6b102f4d24f0c7f8b58c0fab97fff6 Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Wed, 24 Oct 2018 13:46:26 +0800 Subject: [PATCH 202/359] [FLINK-10543][table] Leverage efficient timer deletion in relational operators This closes #6918 --- .../runtime/aggregate/CleanupState.scala | 57 ++++ .../CoProcessFunctionWithCleanupState.scala | 69 ++++ .../aggregate/GroupAggProcessFunction.scala | 4 +- ...KeyedProcessFunctionWithCleanupState.scala | 38 +-- .../aggregate/ProcTimeBoundedRangeOver.scala | 29 +- .../aggregate/ProcTimeBoundedRowsOver.scala | 4 +- .../aggregate/ProcTimeUnboundedOver.scala | 4 +- .../ProcessFunctionWithCleanupState.scala | 41 +-- .../aggregate/RowTimeBoundedRangeOver.scala | 20 +- .../aggregate/RowTimeBoundedRowsOver.scala | 8 +- .../aggregate/RowTimeUnboundedOver.scala | 8 +- .../runtime/join/NonWindowFullJoin.scala | 3 +- ...nWindowFullJoinWithNonEquiPredicates.scala | 23 +- .../runtime/join/NonWindowInnerJoin.scala | 3 +- .../table/runtime/join/NonWindowJoin.scala | 104 +----- .../runtime/join/NonWindowLeftRightJoin.scala | 3 +- ...owLeftRightJoinWithNonEquiPredicates.scala | 25 +- .../runtime/join/NonWindowOuterJoin.scala | 9 +- ...WindowOuterJoinWithNonEquiPredicates.scala | 47 +-- .../triggers/StateCleaningCountTrigger.scala | 13 +- .../runtime/harness/JoinHarnessTest.scala | 320 ++++++++++-------- ...StateCleaningCountTriggerHarnessTest.scala | 7 +- ...dProcessFunctionWithCleanupStateTest.scala | 10 +- .../ProcessFunctionWithCleanupStateTest.scala | 4 +- 24 files changed, 423 insertions(+), 430 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala new file mode 100644 index 0000000000000..d9c8e2ccaee35 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala @@ -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.table.runtime.aggregate + +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import java.lang.{Long => JLong} + +import org.apache.flink.streaming.api.TimerService + +/** + * Base class for clean up state, both for [[ProcessFunction]] and [[CoProcessFunction]]. + */ +trait CleanupState { + + def registerProcessingCleanupTimer( + cleanupTimeState: ValueState[JLong], + currentTime: Long, + minRetentionTime: Long, + maxRetentionTime: Long, + timerService: TimerService): Unit = { + + // last registered timer + val curCleanupTime = cleanupTimeState.value() + + // check if a cleanup timer is registered and + // that the current cleanup timer won't delete state we need to keep + if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) { + // we need to register a new (later) timer + val cleanupTime = currentTime + maxRetentionTime + // register timer and remember clean-up time + timerService.registerProcessingTimeTimer(cleanupTime) + // delete expired timer + if (curCleanupTime != null) { + timerService.deleteProcessingTimeTimer(curCleanupTime) + } + cleanupTimeState.update(cleanupTime) + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala new file mode 100644 index 0000000000000..0c7663621ba58 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala @@ -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.table.runtime.aggregate + +import java.lang.{Long => JLong} + +import org.apache.flink.api.common.state.{State, ValueState, ValueStateDescriptor} +import org.apache.flink.streaming.api.TimeDomain +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.table.api.{StreamQueryConfig, Types} + +abstract class CoProcessFunctionWithCleanupState[IN1, IN2, OUT](queryConfig: StreamQueryConfig) + extends CoProcessFunction[IN1, IN2, OUT] + with CleanupState { + + protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime + protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime + protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 + + // holds the latest registered cleanup timer + private var cleanupTimeState: ValueState[JLong] = _ + + protected def initCleanupTimeState(stateName: String) { + if (stateCleaningEnabled) { + val cleanupTimeDescriptor: ValueStateDescriptor[JLong] = + new ValueStateDescriptor[JLong](stateName, Types.LONG) + cleanupTimeState = getRuntimeContext.getState(cleanupTimeDescriptor) + } + } + + protected def processCleanupTimer( + ctx: CoProcessFunction[IN1, IN2, OUT]#Context, + currentTime: Long): Unit = { + if (stateCleaningEnabled) { + registerProcessingCleanupTimer( + cleanupTimeState, + currentTime, + minRetentionTime, + maxRetentionTime, + ctx.timerService() + ) + } + } + + protected def isProcessingTimeTimer(ctx: OnTimerContext): Boolean = { + ctx.timeDomain() == TimeDomain.PROCESSING_TIME + } + + protected def cleanupState(states: State*): Unit = { + // clear all state + states.foreach(_.clear()) + this.cleanupTimeState.clear() + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index c59efe250c3d2..2d72e6de9616c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -86,7 +86,7 @@ class GroupAggProcessFunction( val currentTime = ctx.timerService().currentProcessingTime() // register state-cleanup timer - registerProcessingCleanupTimer(ctx, currentTime) + processCleanupTimer(ctx, currentTime) val input = inputC.row @@ -172,7 +172,7 @@ class GroupAggProcessFunction( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { cleanupState(state, cntState) function.cleanup() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala index 4d6840a3f43e0..edf5c2cd101c9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala @@ -25,13 +25,15 @@ import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFu import org.apache.flink.table.api.{StreamQueryConfig, Types} abstract class KeyedProcessFunctionWithCleanupState[K, I, O](queryConfig: StreamQueryConfig) - extends KeyedProcessFunction[K, I, O] { + extends KeyedProcessFunction[K, I, O] + with CleanupState { + protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 // holds the latest registered cleanup timer - private var cleanupTimeState: ValueState[JLong] = _ + protected var cleanupTimeState: ValueState[JLong] = _ protected def initCleanupTimeState(stateName: String) { if (stateCleaningEnabled) { @@ -41,23 +43,17 @@ abstract class KeyedProcessFunctionWithCleanupState[K, I, O](queryConfig: Stream } } - protected def registerProcessingCleanupTimer( + protected def processCleanupTimer( ctx: KeyedProcessFunction[K, I, O]#Context, currentTime: Long): Unit = { if (stateCleaningEnabled) { - - // last registered timer - val curCleanupTime = cleanupTimeState.value() - - // check if a cleanup timer is registered and - // that the current cleanup timer won't delete state we need to keep - if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) { - // we need to register a new (later) timer - val cleanupTime = currentTime + maxRetentionTime - // register timer and remember clean-up time - ctx.timerService().registerProcessingTimeTimer(cleanupTime) - cleanupTimeState.update(cleanupTime) - } + registerProcessingCleanupTimer( + cleanupTimeState, + currentTime, + minRetentionTime, + maxRetentionTime, + ctx.timerService() + ) } } @@ -65,16 +61,6 @@ abstract class KeyedProcessFunctionWithCleanupState[K, I, O](queryConfig: Stream ctx.timeDomain() == TimeDomain.PROCESSING_TIME } - protected def needToCleanupState(timestamp: Long): Boolean = { - if (stateCleaningEnabled) { - val cleanupTime = cleanupTimeState.value() - // check that the triggered timer is the last registered processing time timer. - null != cleanupTime && timestamp == cleanupTime - } else { - false - } - } - protected def cleanupState(states: State*): Unit = { // clear all state states.foreach(_.clear()) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala index 591b942571f19..6126dc73b3085 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala @@ -95,7 +95,7 @@ class ProcTimeBoundedRangeOver( val currentTime = ctx.timerService.currentProcessingTime // register state-cleanup timer - registerProcessingCleanupTimer(ctx, currentTime) + processCleanupTimer(ctx, currentTime) // buffer the event incoming event @@ -117,11 +117,14 @@ class ProcTimeBoundedRangeOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (needToCleanupState(timestamp)) { - // clean up and return - cleanupState(rowMapState, accumulatorState) - function.cleanup() - return + if (stateCleaningEnabled) { + val cleanupTime = cleanupTimeState.value() + if (null != cleanupTime && timestamp == cleanupTime) { + // clean up and return + cleanupState(rowMapState, accumulatorState) + function.cleanup() + return + } } // remove timestamp set outside of ProcessFunction. @@ -131,11 +134,10 @@ class ProcTimeBoundedRangeOver( // that have registered this time trigger 1 ms ago val currentTime = timestamp - 1 - var i = 0 // get the list of elements of current proctime val currentElements = rowMapState.get(currentTime) - // Expired clean-up timers pass the needToCleanupState() check. + // Expired clean-up timers pass the needToCleanupState check. // Perform a null check to verify that we have data to process. if (null == currentElements) { return @@ -156,7 +158,6 @@ class ProcTimeBoundedRangeOver( // and eliminate them. Multiple elements could have been received at the same timestamp // the removal of old elements happens only once per proctime as onTimer is called only once val iter = rowMapState.iterator - val markToRemove = new ArrayList[Long]() while (iter.hasNext) { val entry = iter.next() val elementKey = entry.getKey @@ -169,17 +170,9 @@ class ProcTimeBoundedRangeOver( function.retract(accumulators, retractRow) iRemove += 1 } - // mark element for later removal not to modify the iterator over MapState - markToRemove.add(elementKey) + iter.remove() } } - // need to remove in 2 steps not to have concurrent access errors via iterator to the MapState - i = 0 - while (i < markToRemove.size()) { - rowMapState.remove(markToRemove.get(i)) - i += 1 - } - // add current elements to aggregator. Multiple elements might // have arrived in the same proctime diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala index ccddaa5b10fc7..fa58ac50529ec 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala @@ -110,7 +110,7 @@ class ProcTimeBoundedRowsOver( val currentTime = ctx.timerService.currentProcessingTime // register state-cleanup timer - registerProcessingCleanupTimer(ctx, currentTime) + processCleanupTimer(ctx, currentTime) // initialize state for the processed element var accumulators = accumulatorState.value @@ -187,7 +187,7 @@ class ProcTimeBoundedRowsOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { cleanupState(rowMapState, accumulatorState, counterState, smallestTsState) function.cleanup() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala index 6e4c510578682..ce1a95971a9ef 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala @@ -71,7 +71,7 @@ class ProcTimeUnboundedOver( out: Collector[CRow]): Unit = { // register state-cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) val input = inputC.row @@ -95,7 +95,7 @@ class ProcTimeUnboundedOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { cleanupState(state) function.cleanup() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala index 292fd3bdf1e27..7263de72c4b28 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala @@ -26,40 +26,35 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryConfig) - extends ProcessFunction[IN, OUT]{ + extends ProcessFunction[IN, OUT] + with CleanupState { protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 // holds the latest registered cleanup timer - private var cleanupTimeState: ValueState[JLong] = _ + protected var cleanupTimeState: ValueState[JLong] = _ protected def initCleanupTimeState(stateName: String) { if (stateCleaningEnabled) { - val inputCntDescriptor: ValueStateDescriptor[JLong] = + val cleanupTimeDescriptor: ValueStateDescriptor[JLong] = new ValueStateDescriptor[JLong](stateName, Types.LONG) - cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor) + cleanupTimeState = getRuntimeContext.getState(cleanupTimeDescriptor) } } - protected def registerProcessingCleanupTimer( + protected def processCleanupTimer( ctx: ProcessFunction[IN, OUT]#Context, currentTime: Long): Unit = { if (stateCleaningEnabled) { - - // last registered timer - val curCleanupTime = cleanupTimeState.value() - - // check if a cleanup timer is registered and - // that the current cleanup timer won't delete state we need to keep - if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) { - // we need to register a new (later) timer - val cleanupTime = currentTime + maxRetentionTime - // register timer and remember clean-up time - ctx.timerService().registerProcessingTimeTimer(cleanupTime) - cleanupTimeState.update(cleanupTime) - } + registerProcessingCleanupTimer( + cleanupTimeState, + currentTime, + minRetentionTime, + maxRetentionTime, + ctx.timerService() + ) } } @@ -67,16 +62,6 @@ abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryC ctx.timeDomain() == TimeDomain.PROCESSING_TIME } - protected def needToCleanupState(timestamp: Long): Boolean = { - if (stateCleaningEnabled) { - val cleanupTime = cleanupTimeState.value() - // check that the triggered timer is the last registered processing time timer. - null != cleanupTime && timestamp == cleanupTime - } else { - false - } - } - protected def cleanupState(states: State*): Unit = { // clear all state states.foreach(_.clear()) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala index b13acdf43ccab..7c509d699e8d0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala @@ -114,7 +114,7 @@ class RowTimeBoundedRangeOver( val input = inputC.row // register state-cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation val triggeringTs = input.getField(rowTimeIdx).asInstanceOf[Long] @@ -143,7 +143,7 @@ class RowTimeBoundedRangeOver( out: Collector[CRow]): Unit = { if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { val keysIt = dataState.keys.iterator() val lastProcessedTime = lastTriggeringTsState.value @@ -164,7 +164,7 @@ class RowTimeBoundedRangeOver( // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. // We leave the state as it is and schedule a new cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } } return @@ -188,9 +188,6 @@ class RowTimeBoundedRangeOver( aggregatesIndex = 0 } - // keep up timestamps of retract data - val retractTsList: JList[Long] = new JArrayList[Long] - // do retraction val iter = dataState.iterator() while (iter.hasNext) { @@ -205,7 +202,7 @@ class RowTimeBoundedRangeOver( function.retract(accumulators, retractRow) dataListIndex += 1 } - retractTsList.add(dataTs) + iter.remove() } } @@ -230,20 +227,13 @@ class RowTimeBoundedRangeOver( dataListIndex += 1 } - // remove the data that has been retracted - dataListIndex = 0 - while (dataListIndex < retractTsList.size) { - dataState.remove(retractTsList.get(dataListIndex)) - dataListIndex += 1 - } - // update state accumulatorState.update(accumulators) } lastTriggeringTsState.update(timestamp) // update cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } override def close(): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index e120d6b0afda4..d01a499e88efc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala @@ -123,7 +123,7 @@ class RowTimeBoundedRowsOver( val input = inputC.row // register state-cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation val triggeringTs = input.getField(rowTimeIdx).asInstanceOf[Long] @@ -152,7 +152,7 @@ class RowTimeBoundedRowsOver( out: Collector[CRow]): Unit = { if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { val keysIt = dataState.keys.iterator() val lastProcessedTime = lastTriggeringTsState.value @@ -173,7 +173,7 @@ class RowTimeBoundedRowsOver( // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. // We leave the state as it is and schedule a new cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } } return @@ -263,7 +263,7 @@ class RowTimeBoundedRowsOver( lastTriggeringTsState.update(timestamp) // update cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } override def close(): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index 181c7680a35a4..690d0d05ee304 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -108,7 +108,7 @@ abstract class RowTimeUnboundedOver( val input = inputC.row // register state-cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) val timestamp = input.getField(rowTimeIdx).asInstanceOf[Long] val curWatermark = ctx.timerService().currentWatermark() @@ -143,7 +143,7 @@ abstract class RowTimeUnboundedOver( out: Collector[CRow]): Unit = { if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { // we check whether there are still records which have not been processed yet val noRecordsToProcess = !rowMapState.keys.iterator().hasNext @@ -155,7 +155,7 @@ abstract class RowTimeUnboundedOver( // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. // We leave the state as it is and schedule a new cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } } return @@ -207,7 +207,7 @@ abstract class RowTimeUnboundedOver( } // update cleanup timer - registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala index 57c60f179c644..5b1069ff28bc3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala @@ -66,13 +66,12 @@ class NonWindowFullJoin( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val inputRow = value.row - updateCurrentSide(value, ctx, timerState, currentSideState) + updateCurrentSide(value, ctx, currentSideState) cRowWrapper.reset() cRowWrapper.setCollector(out) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala index 9c27eb461a689..0166eef796149 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala @@ -68,14 +68,13 @@ class NonWindowFullJoinWithNonEquiPredicates( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val currentJoinCntState = getJoinCntState(joinCntState, recordFromLeft) val inputRow = value.row - val cntAndExpiredTime = updateCurrentSide(value, ctx, timerState, currentSideState) + val cntAndExpiredTime = updateCurrentSide(value, ctx, currentSideState) if (!value.change && cntAndExpiredTime.f0 <= 0) { currentJoinCntState.remove(inputRow) } @@ -99,18 +98,18 @@ class NonWindowFullJoinWithNonEquiPredicates( } /** - * Removes records which are expired from left state. Register a new timer if the state still - * holds records after the clean-up. Also, clear leftJoinCnt map state when clear left - * rowMapState. + * Called when a processing timer trigger. + * Expire left/right expired records and expired joinCnt state. */ - override def expireOutTimeRow( - curTime: Long, - rowMapState: MapState[Row, JTuple2[Long, Long]], - timerState: ValueState[Long], - isLeft: Boolean, - ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { + override def onTimer( + timestamp: Long, + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { - expireOutTimeRow(curTime, rowMapState, timerState, isLeft, joinCntState, ctx) + // expired timer has already been removed, delete state directly. + if (stateCleaningEnabled) { + cleanupState(leftState, rightState, joinCntState(0), joinCntState(1)) + } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala index 2e5832c269424..91a75077cd885 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala @@ -63,13 +63,12 @@ class NonWindowInnerJoin( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], isLeft: Boolean): Unit = { val inputRow = value.row - updateCurrentSide(value, ctx, timerState, currentSideState) + updateCurrentSide(value, ctx, currentSideState) cRowWrapper.setCollector(out) cRowWrapper.setChange(value.change) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala index c59f4c2f44b04..e15cbfa550c39 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.join import org.apache.flink.api.common.functions.FlatJoinFunction import org.apache.flink.api.common.functions.util.FunctionUtils -import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, ValueState, ValueStateDescriptor} +import org.apache.flink.api.common.state.{MapState, MapStateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.api.java.typeutils.TupleTypeInfo @@ -27,6 +27,7 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.co.CoProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.runtime.aggregate.CoProcessFunctionWithCleanupState import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.typeutils.TypeCheckUtils._ import org.apache.flink.table.util.Logging @@ -48,7 +49,7 @@ abstract class NonWindowJoin( genJoinFuncName: String, genJoinFuncCode: String, queryConfig: StreamQueryConfig) - extends CoProcessFunction[CRow, CRow, CRow] + extends CoProcessFunctionWithCleanupState[CRow, CRow, CRow](queryConfig) with Compiler[FlatJoinFunction[Row, Row, Row]] with Logging { @@ -62,15 +63,6 @@ abstract class NonWindowJoin( protected var rightState: MapState[Row, JTuple2[Long, Long]] = _ protected var cRowWrapper: CRowWrappingMultiOutputCollector = _ - protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime - protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime - protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 - - // state to record last timer of left stream, 0 means no timer - protected var leftTimer: ValueState[Long] = _ - // state to record last timer of right stream, 0 means no timer - protected var rightTimer: ValueState[Long] = _ - // other condition function protected var joinFunction: FlatJoinFunction[Row, Row, Row] = _ @@ -78,7 +70,8 @@ abstract class NonWindowJoin( protected var curProcessTime: Long = _ override def open(parameters: Configuration): Unit = { - LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n Code:\n$genJoinFuncCode") + LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " + + s"Code:\n$genJoinFuncCode") val clazz = compile( getRuntimeContext.getUserCodeClassLoader, genJoinFuncName, @@ -100,10 +93,7 @@ abstract class NonWindowJoin( rightState = getRuntimeContext.getMapState(rightStateDescriptor) // initialize timer state - val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long]) - leftTimer = getRuntimeContext.getState(valueStateDescriptor1) - val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long]) - rightTimer = getRuntimeContext.getState(valueStateDescriptor2) + initCleanupTimeState("NonWindowJoinCleanupTime") cRowWrapper = new CRowWrappingMultiOutputCollector() LOG.debug("Instantiating NonWindowJoin.") @@ -122,7 +112,7 @@ abstract class NonWindowJoin( ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow]): Unit = { - processElement(valueC, ctx, out, leftTimer, leftState, rightState, isLeft = true) + processElement(valueC, ctx, out, leftState, rightState, isLeft = true) } /** @@ -138,7 +128,7 @@ abstract class NonWindowJoin( ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow]): Unit = { - processElement(valueC, ctx, out, rightTimer, rightState, leftState, isLeft = false) + processElement(valueC, ctx, out, rightState, leftState, isLeft = false) } /** @@ -154,28 +144,13 @@ abstract class NonWindowJoin( ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (stateCleaningEnabled && leftTimer.value == timestamp) { - expireOutTimeRow( - timestamp, - leftState, - leftTimer, - isLeft = true, - ctx - ) - } - - if (stateCleaningEnabled && rightTimer.value == timestamp) { - expireOutTimeRow( - timestamp, - rightState, - rightTimer, - isLeft = false, - ctx - ) + // expired timer has already been removed, delete state directly. + if (stateCleaningEnabled) { + cleanupState(leftState, rightState) } } - def getNewExpiredTime(curProcessTime: Long, oldExpiredTime: Long): Long = { + protected def getNewExpiredTime(curProcessTime: Long, oldExpiredTime: Long): Long = { if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) { curProcessTime + maxRetentionTime } else { @@ -183,53 +158,15 @@ abstract class NonWindowJoin( } } - /** - * Removes records which are expired from the state. Register a new timer if the state still - * holds records after the clean-up. - */ - def expireOutTimeRow( - curTime: Long, - rowMapState: MapState[Row, JTuple2[Long, Long]], - timerState: ValueState[Long], - isLeft: Boolean, - ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { - - val rowMapIter = rowMapState.iterator() - var validTimestamp: Boolean = false - - while (rowMapIter.hasNext) { - val mapEntry = rowMapIter.next() - val recordExpiredTime = mapEntry.getValue.f1 - if (recordExpiredTime <= curTime) { - rowMapIter.remove() - } else { - // we found a timestamp that is still valid - validTimestamp = true - } - } - - // If the state has non-expired timestamps, register a new timer. - // Otherwise clean the complete state for this input. - if (validTimestamp) { - val cleanupTime = curTime + maxRetentionTime - ctx.timerService.registerProcessingTimeTimer(cleanupTime) - timerState.update(cleanupTime) - } else { - timerState.clear() - rowMapState.clear() - } - } - /** * Puts or Retract an element from the input stream into state and search the other state to * output records meet the condition. Records will be expired in state if state retention time * has been specified. */ - def processElement( + protected def processElement( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], isLeft: Boolean): Unit @@ -240,14 +177,12 @@ abstract class NonWindowJoin( * * @param value The input CRow * @param ctx The ctx to register timer or get current time - * @param timerState The state to record last timer * @param currentSideState The state to hold current side stream element * @return The row number and expired time for current input row */ - def updateCurrentSide( + protected def updateCurrentSide( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]]): JTuple2[Long, Long] = { val inputRow = value.row @@ -261,10 +196,7 @@ abstract class NonWindowJoin( cntAndExpiredTime.f1 = getNewExpiredTime(curProcessTime, cntAndExpiredTime.f1) // update timer if necessary - if (stateCleaningEnabled && timerState.value() == 0) { - timerState.update(cntAndExpiredTime.f1) - ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime.f1) - } + processCleanupTimer(ctx, curProcessTime) // update current side stream state if (!value.change) { @@ -282,7 +214,7 @@ abstract class NonWindowJoin( cntAndExpiredTime } - def callJoinFunction( + protected def callJoinFunction( inputRow: Row, inputRowFromLeft: Boolean, otherSideRow: Row, @@ -294,8 +226,4 @@ abstract class NonWindowJoin( joinFunction.join(otherSideRow, inputRow, cRowWrapper) } } - - override def close(): Unit = { - FunctionUtils.closeFunction(joinFunction) - } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala index b4f14e494f7d5..5995fb81a2c34 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala @@ -69,13 +69,12 @@ class NonWindowLeftRightJoin( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val inputRow = value.row - updateCurrentSide(value, ctx, timerState, currentSideState) + updateCurrentSide(value, ctx, currentSideState) cRowWrapper.reset() cRowWrapper.setCollector(out) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala index 33517cca5f91f..a3e25f9489ddb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala @@ -71,14 +71,13 @@ class NonWindowLeftRightJoinWithNonEquiPredicates( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], - timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val currentJoinCntState = getJoinCntState(joinCntState, recordFromLeft) val inputRow = value.row - val cntAndExpiredTime = updateCurrentSide(value, ctx, timerState, currentSideState) + val cntAndExpiredTime = updateCurrentSide(value, ctx, currentSideState) if (!value.change && cntAndExpiredTime.f0 <= 0 && recordFromLeft == isLeftJoin) { currentJoinCntState.remove(inputRow) } @@ -101,17 +100,21 @@ class NonWindowLeftRightJoinWithNonEquiPredicates( } /** - * Removes records which are expired from state. Register a new timer if the state still - * holds records after the clean-up. Also, clear joinCnt map state when clear rowMapState. + * Called when a processing timer trigger. + * Expire left/right expired records and expired joinCnt state. */ - override def expireOutTimeRow( - curTime: Long, - rowMapState: MapState[Row, JTuple2[Long, Long]], - timerState: ValueState[Long], - isLeft: Boolean, - ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { + override def onTimer( + timestamp: Long, + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { - expireOutTimeRow(curTime, rowMapState, timerState, isLeft, joinCntState, ctx) + // expired timer has already been removed, delete state directly. + if (stateCleaningEnabled) { + cleanupState( + leftState, + rightState, + getJoinCntState(joinCntState, isLeftJoin)) + } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala index 0018a16bc23ac..9fadbb0eb6441 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala @@ -73,7 +73,7 @@ abstract class NonWindowOuterJoin( * @param otherSideState the other side state * @return the number of matched rows */ - def preservedJoin( + protected def preservedJoin( inputRow: Row, inputRowFromLeft: Boolean, otherSideState: MapState[Row, JTuple2[Long, Long]]): Long = { @@ -106,7 +106,7 @@ abstract class NonWindowOuterJoin( * RowWrapper has been reset before we call retractJoin and we also assume that the current * change of cRowWrapper is equal to value.change. */ - def retractJoin( + protected def retractJoin( value: CRow, inputRowFromLeft: Boolean, currentSideState: MapState[Row, JTuple2[Long, Long]], @@ -152,7 +152,8 @@ abstract class NonWindowOuterJoin( * Return approximate number of records in corresponding state. Only check if record number is * 0, 1 or bigger. */ - def approxiRecordNumInState(currentSideState: MapState[Row, JTuple2[Long, Long]]): Long = { + protected def approxiRecordNumInState( + currentSideState: MapState[Row, JTuple2[Long, Long]]): Long = { var recordNum = 0L val it = currentSideState.iterator() while(it.hasNext && recordNum < 2) { @@ -164,7 +165,7 @@ abstract class NonWindowOuterJoin( /** * Append input row with default null value if there is no match and Collect. */ - def collectAppendNull( + protected def collectAppendNull( inputRow: Row, inputFromLeft: Boolean, out: Collector[Row]): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala index 8fe2f4fe41082..0eb6a8114fd76 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala @@ -21,7 +21,6 @@ import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.configuration.Configuration -import org.apache.flink.streaming.api.functions.co.CoProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.runtime.types.CRow import org.apache.flink.types.Row @@ -83,7 +82,7 @@ import org.apache.flink.types.Row * unmatched or vice versa. The RowWrapper has been reset before we call retractJoin and we * also assume that the current change of cRowWrapper is equal to value.change. */ - def retractJoinWithNonEquiPreds( + protected def retractJoinWithNonEquiPreds( value: CRow, inputRowFromLeft: Boolean, otherSideState: MapState[Row, JTuple2[Long, Long]], @@ -131,48 +130,6 @@ import org.apache.flink.types.Row } } - /** - * Removes records which are expired from state. Registers a new timer if the state still - * holds records after the clean-up. Also, clear joinCnt map state when clear rowMapState. - */ - def expireOutTimeRow( - curTime: Long, - rowMapState: MapState[Row, JTuple2[Long, Long]], - timerState: ValueState[Long], - isLeft: Boolean, - joinCntState: Array[MapState[Row, Long]], - ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { - - val currentJoinCntState = getJoinCntState(joinCntState, isLeft) - val rowMapIter = rowMapState.iterator() - var validTimestamp: Boolean = false - - while (rowMapIter.hasNext) { - val mapEntry = rowMapIter.next() - val recordExpiredTime = mapEntry.getValue.f1 - if (recordExpiredTime <= curTime) { - rowMapIter.remove() - currentJoinCntState.remove(mapEntry.getKey) - } else { - // we found a timestamp that is still valid - validTimestamp = true - } - } - // If the state has non-expired timestamps, register a new timer. - // Otherwise clean the complete state for this input. - if (validTimestamp) { - val cleanupTime = curTime + maxRetentionTime - ctx.timerService.registerProcessingTimeTimer(cleanupTime) - timerState.update(cleanupTime) - } else { - timerState.clear() - rowMapState.clear() - if (isLeft == isLeftJoin) { - currentJoinCntState.clear() - } - } - } - /** * Get left or right join cnt state. * @@ -181,7 +138,7 @@ import org.apache.flink.types.Row * @param isLeftCntState the flag whether get the left join cnt state * @return the corresponding join cnt state */ - def getJoinCntState( + protected def getJoinCntState( joinCntState: Array[MapState[Row, Long]], isLeftCntState: Boolean) : MapState[Row, Long] = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala index 3c18449e95321..6ae5e6340a635 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala @@ -71,6 +71,10 @@ class StateCleaningCountTrigger(queryConfig: StreamQueryConfig, maxCount: Long) val cleanupTime = currentTime + maxRetentionTime // register timer and remember clean-up time ctx.registerProcessingTimeTimer(cleanupTime) + // delete expired timer + if (curCleanupTime != null) { + ctx.deleteProcessingTimeTimer(curCleanupTime) + } ctx.getPartitionedState(cleanupStateDesc).update(cleanupTime) } @@ -93,12 +97,9 @@ class StateCleaningCountTrigger(queryConfig: StreamQueryConfig, maxCount: Long) ctx: TriggerContext): TriggerResult = { if (stateCleaningEnabled) { - val cleanupTime = ctx.getPartitionedState(cleanupStateDesc).value() - // check that the triggered timer is the last registered processing time timer. - if (null != cleanupTime && time == cleanupTime) { - clear(window, ctx) - return TriggerResult.FIRE_AND_PURGE - } + // do clear directly, since we have already deleted expired timer + clear(window, ctx) + return TriggerResult.FIRE_AND_PURGE } TriggerResult.CONTINUE } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala index bd19be837574c..4619c759c31af 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala @@ -21,21 +21,18 @@ import java.lang.{Integer => JInt, Long => JLong} import java.util.concurrent.ConcurrentLinkedQueue import org.apache.flink.api.common.time.Time -import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.java.operators.join.JoinType -import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness -import org.apache.flink.table.api.{StreamQueryConfig, Types} -import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, RowResultSortComparatorWithWatermarks, TestStreamQueryConfig, TupleRowKeySelector} +import org.apache.flink.table.api.Types +import org.apache.flink.table.runtime.harness.HarnessTestBase.{TestStreamQueryConfig, TupleRowKeySelector} import org.apache.flink.table.runtime.join._ import org.apache.flink.table.runtime.operators.KeyedCoProcessOperatorWithWatermarkDelay -import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.apache.flink.types.Row -import org.junit.Assert.{assertEquals, assertTrue} +import org.apache.flink.table.runtime.types.CRow +import org.junit.Assert.assertEquals import org.junit.Test /** @@ -830,14 +827,6 @@ class JoinHarnessTest extends HarnessTestBase { @Test def testNonWindowInnerJoin() { - val joinReturnType = CRowTypeInfo(new RowTypeInfo( - Array[TypeInformation[_]]( - INT_TYPE_INFO, - STRING_TYPE_INFO, - INT_TYPE_INFO, - STRING_TYPE_INFO), - Array("a", "b", "c", "d"))) - val joinProcessFunc = new NonWindowInnerJoin( rowType, rowType, @@ -879,35 +868,32 @@ class JoinHarnessTest extends HarnessTestBase { // right stream input and output normally testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi1"))) - assertEquals(6, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(1) timer_key_time(1:5, 2:6) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello1"))) - assertEquals(8, testHarness.numKeyedStateEntries()) - assertEquals(4, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) + assertEquals(6, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired left stream record with key value of 1 + // expired stream record with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi2"))) - assertEquals(6, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) - - // expired all left stream record - testHarness.setProcessingTime(6) - assertEquals(4, testHarness.numKeyedStateEntries()) + // lkeys(2) rkeys(1, 2) timer_key_time(1:9, 2:6) + assertEquals(5, testHarness.numKeyedStateEntries()) assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired right stream record with key value of 2 - testHarness.setProcessingTime(8) + // expired all left stream records + testHarness.setProcessingTime(6) + // lkeys() rkeys(1) timer_key_time(1:9) assertEquals(2, testHarness.numKeyedStateEntries()) assertEquals(1, testHarness.numProcessingTimeTimers()) - testHarness.setProcessingTime(10) - assertTrue(testHarness.numKeyedStateEntries() > 0) - // expired all right stream record - testHarness.setProcessingTime(11) + // expired all stream records + testHarness.setProcessingTime(9) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -975,32 +961,37 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys() timer_key_time(1:5, 2:6) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello1"))) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(4, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "aaa"))) - // expired left stream record with key value of 1 + // expired stream records with key value of 1 testHarness.setProcessingTime(5) + // lkeys(2) rkeys(2) timer_key_time(2:6) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi2"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi2"))) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(2) rkeys(2) timer_key_time(1:9, 2:6) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all left stream record + // expired all stream records testHarness.setProcessingTime(6) - assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // lkeys() rkeys() timer_key_time(1:9) + assertEquals(1, testHarness.numKeyedStateEntries()) + assertEquals(1, testHarness.numProcessingTimeTimers()) - // expired right stream record with key value of 2 - testHarness.setProcessingTime(8) + // expired all data + testHarness.setProcessingTime(9) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1067,32 +1058,36 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys() timer_key_time(1:5, 2:6) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello1"))) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(4, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "aaa"))) - // expired left stream record with key value of 1 + // expired stream records with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi2"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi2"))) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(2) rkeys(2) timer_key_time(1:9, 2:6) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all left stream record + // expired stream records with key value of 2 testHarness.setProcessingTime(6) - assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // lkeys() rkeys() timer_key_time(1:9) + assertEquals(1, testHarness.numKeyedStateEntries()) + assertEquals(1, testHarness.numProcessingTimeTimers()) - // expired right stream record with key value of 2 - testHarness.setProcessingTime(8) + // expired all data + testHarness.setProcessingTime(9) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1160,7 +1155,7 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement1(new StreamRecord( CRow(1: JInt, "bbb"))) assertEquals(1, testHarness.numProcessingTimeTimers()) - // 1 left timer(5), 1 left key(1), 1 join cnt + // lkeys(1) rkeys() timer_key_time(1:5) assertEquals(3, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) testHarness.processElement1(new StreamRecord( @@ -1168,7 +1163,8 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement1(new StreamRecord( CRow(2: JInt, "bbb"))) assertEquals(2, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,6), 2 left key(1,2), 2 join cnt + // lkeys(1, 2) rkeys() timer_key_time(1:5, 2:6) + // l_join_cnt_keys(1, 2) assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1177,17 +1173,19 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "bbb"))) - // 2 left timer(5,6), 2 left keys(1,2), 2 join cnt, 1 right timer(7), 1 right key(1) - assertEquals(8, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(1) timer_key_time(1:5, 2:6) + // l_join_cnt_keys(1, 2) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "ccc"))) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello"))) - // 2 left timer(5,6), 2 left keys(1,2), 2 join cnt, 2 right timer(7,8), 2 right key(1,2) - assertEquals(10, testHarness.numKeyedStateEntries()) - assertEquals(4, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) + // l_join_cnt_keys(1, 2) + assertEquals(8, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "aaa"))) @@ -1197,22 +1195,29 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "Hi2"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // expired left stream record with key value of 1 + // lkeys(1, 2) rkeys(2) timer_key_time(1:8, 2:6) + // l_join_cnt_keys(1, 2) + assertEquals(7, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(5) + // [1]. this will clean up left stream records with expired time of 5 testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi3"))) + // [2]. there are no elements can be connected, since be cleaned by [1] testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi3"))) - // 1 left timer(6), 1 left keys(2), 1 join cnt, 2 right timer(7,8), 1 right key(2) - assertEquals(6, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(2) timer_key_time(1:8, 2:6) + // l_join_cnt_keys(1, 2) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all left stream record + // expired all records with key value of 2 testHarness.setProcessingTime(6) + // lkeys(1) rkeys() timer_key_time(1:8) + // l_join_cnt_keys(1) assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(1, testHarness.numProcessingTimeTimers()) - // expired right stream record with key value of 2 + // expired all data testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1253,6 +1258,12 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "bbb", 1: JInt, "Hi1"))) expectedOutput.add(new StreamRecord( CRow(1: JInt, "bbb", null: JInt, null))) + // processing time of 5 + // timer of 8, we use only one timer state now + expectedOutput.add(new StreamRecord( + CRow(false, 1: JInt, "bbb", null: JInt, null))) + expectedOutput.add(new StreamRecord( + CRow(1: JInt, "bbb", 1: JInt, "Hi3"))) verify(expectedOutput, result) testHarness.close() @@ -1305,32 +1316,36 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys() rkeys(1, 2) timer_key_time(1:5, 2:6) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "Hello1"))) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(4, testHarness.numProcessingTimeTimers()) + // lkeys(2) rkeys(1, 2) timer_key_time(1:5, 2:6) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "aaa"))) - // expired right stream record with key value of 1 + // expired stream records with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "Hi2"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi2"))) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(2) rkeys(2) timer_key_time(1:9, 2:6) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all right stream record + // expired stream records with key value of 2 testHarness.setProcessingTime(6) - assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // lkeys() rkeys() timer_key_time(1:9) + assertEquals(1, testHarness.numKeyedStateEntries()) + assertEquals(1, testHarness.numProcessingTimeTimers()) - // expired left stream record with key value of 2 - testHarness.setProcessingTime(8) + // expired all data + testHarness.setProcessingTime(9) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1398,15 +1413,17 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement2(new StreamRecord( CRow(1: JInt, "bbb"))) assertEquals(1, testHarness.numProcessingTimeTimers()) - // 1 right timer(5), 1 right key(1), 1 join cnt + // lkeys() rkeys(1) timer_key_time(1:5) + // r_join_cnt_keys(1) assertEquals(3, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "bbb"))) + // lkeys() rkeys(1, 2) timer_key_time(1:5, 2:6) + // r_join_cnt_keys(1, 2) assertEquals(2, testHarness.numProcessingTimeTimers()) - // 2 right timer(5,6), 2 right key(1,2), 2 join cnt assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1415,17 +1432,19 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "bbb"))) - // 2 right timer(5,6), 2 right keys(1,2), 2 join cnt, 1 left timer(7), 1 left key(1) - assertEquals(8, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(1) rkeys(1, 2) timer_key_time(1:5, 2:6) + // r_join_cnt_keys(1, 2) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "ccc"))) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "Hello"))) - // 2 right timer(5,6), 2 right keys(1,2), 2 join cnt, 2 left timer(7,8), 2 left key(1,2) - assertEquals(10, testHarness.numKeyedStateEntries()) - assertEquals(4, testHarness.numProcessingTimeTimers()) + // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) + // r_join_cnt_keys(1, 2) + assertEquals(8, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "aaa"))) @@ -1435,22 +1454,27 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "Hi2"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // expired right stream record with key value of 1 + // lkeys(2) rkeys(1, 2) timer_key_time(1:8, 2:6) + // r_join_cnt_keys(1, 2) + assertEquals(7, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(5) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "Hi3"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi3"))) - // 1 right timer(6), 1 right keys(2), 1 join cnt, 2 left timer(7,8), 1 left key(2) - assertEquals(6, testHarness.numKeyedStateEntries()) - assertEquals(3, testHarness.numProcessingTimeTimers()) + // lkeys(2) rkeys(1, 2) timer_key_time(1:8, 2:6) + // r_join_cnt_keys(1, 2) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all right stream record + // expired all stream records with key value of 2 + // lkeys() rkeys(1) timer_key_time(1:8) + // r_join_cnt_keys(1) testHarness.setProcessingTime(6) assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(1, testHarness.numProcessingTimeTimers()) - // expired left stream record with key value of 2 + // expired all data testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1491,6 +1515,12 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "Hi1", 1: JInt, "bbb"))) expectedOutput.add(new StreamRecord( CRow(null: JInt, null, 1: JInt, "bbb"))) + // processing time of 5 + // timer of 8, we use only one timer state now + expectedOutput.add(new StreamRecord( + CRow(false, null: JInt, null, 1: JInt, "bbb"))) + expectedOutput.add(new StreamRecord( + CRow(1: JInt, "Hi3", 1: JInt, "bbb"))) verify(expectedOutput, result) testHarness.close() @@ -1524,8 +1554,8 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "bbb"))) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "ccc"))) + // lkeys(1) rkeys() timer_key_time(1:5) assertEquals(1, testHarness.numProcessingTimeTimers()) - // 1 left timer(5), 1 left key(1) assertEquals(2, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) @@ -1534,8 +1564,7 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement2(new StreamRecord( CRow(2: JInt, "ccc"))) assertEquals(2, testHarness.numProcessingTimeTimers()) - // 1 left timer(5), 1 left key(1) - // 1 right timer(6), 1 right key(1) + // lkeys(1) rkeys(2) timer_key_time(1:5, 2:6) assertEquals(4, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1543,18 +1572,16 @@ class JoinHarnessTest extends HarnessTestBase { CRow(2: JInt, "aaa"))) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "ddd"))) - assertEquals(3, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,7), 2 left key(1,2) - // 1 right timer(6), 1 right key(1) - assertEquals(6, testHarness.numKeyedStateEntries()) + // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "ddd"))) - assertEquals(4, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,7), 2 left key(1,2) - // 2 right timer(6,7), 2 right key(1,2) - assertEquals(8, testHarness.numKeyedStateEntries()) + // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( @@ -1565,28 +1592,26 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "ddd"))) - assertEquals(4, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,7), 1 left key(1) - // 2 right timer(6,7), 1 right key(2) - assertEquals(6, testHarness.numKeyedStateEntries()) + // lkeys(1) rkeys(2) timer_key_time(1:8, 2:6) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(4, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(5) - assertEquals(3, testHarness.numProcessingTimeTimers()) - // 1 left timer(7) - // 2 right timer(6,7), 1 right key(2) + assertEquals(2, testHarness.numProcessingTimeTimers()) assertEquals(4, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(6) - assertEquals(2, testHarness.numProcessingTimeTimers()) - // 1 left timer(7) - // 2 right timer(7) + // lkeys(1) rkeys() timer_key_time(1:8) + assertEquals(1, testHarness.numProcessingTimeTimers()) assertEquals(2, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(7) - assertEquals(0, testHarness.numProcessingTimeTimers()) - assertEquals(0, testHarness.numKeyedStateEntries()) + assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(2, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(8) + assertEquals(0, testHarness.numProcessingTimeTimers()) + assertEquals(0, testHarness.numKeyedStateEntries()) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "bbb"))) testHarness.processElement2(new StreamRecord( @@ -1693,8 +1718,9 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "bbb"))) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "ccc"))) + // lkeys(1) rkeys() timer_key_time(1:5) + // l_join_cnt_keys(1) r_join_cnt_keys() assertEquals(1, testHarness.numProcessingTimeTimers()) - // 1 left timer(5), 1 left key(1), 1 left joincnt key(1) assertEquals(3, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) @@ -1702,9 +1728,9 @@ class JoinHarnessTest extends HarnessTestBase { CRow(2: JInt, "bbb"))) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "ccc"))) + // lkeys(1) rkeys(2) timer_key_time(1:5, 2:6) + // l_join_cnt_keys(1) r_join_cnt_keys(2) assertEquals(2, testHarness.numProcessingTimeTimers()) - // 1 left timer(5), 1 left key(1), 1 left joincnt key(1) - // 1 right timer(6), 1 right key(1), 1 right joincnt key(1) assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1712,46 +1738,46 @@ class JoinHarnessTest extends HarnessTestBase { CRow(2: JInt, "aaa"))) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "ddd"))) - assertEquals(3, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2) - // 1 right timer(6), 1 right key(1), 1 right joincnt key(1) - assertEquals(9, testHarness.numKeyedStateEntries()) + // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) + // l_join_cnt_keys(1, 2) r_join_cnt_keys(2) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(8, testHarness.numKeyedStateEntries()) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "ddd"))) - assertEquals(4, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2) - // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2) - assertEquals(12, testHarness.numKeyedStateEntries()) + // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) + // l_join_cnt_keys(1, 2) r_join_cnt_keys(1, 2) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(10, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( CRow(false, 2: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "ddd"))) - assertEquals(4, testHarness.numProcessingTimeTimers()) - // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2) - // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2) - assertEquals(12, testHarness.numKeyedStateEntries()) + // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:8, 2:6) + // l_join_cnt_keys(1, 2) r_join_cnt_keys(1, 2) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(10, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(5) - assertEquals(3, testHarness.numProcessingTimeTimers()) - // 1 left timer(7), 1 left key(2), 1 left joincnt key(2) - // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2) - assertEquals(9, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(10, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(6) - assertEquals(2, testHarness.numProcessingTimeTimers()) - // 1 left timer(7), 1 left key(2), 1 left joincnt key(2) - // 1 right timer(7), 1 right key(2), 1 right joincnt key(2) - assertEquals(6, testHarness.numKeyedStateEntries()) + // lkeys(1) rkeys(1) timer_key_time(1:8) + // l_join_cnt_keys(1) r_join_cnt_keys(1) + assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(7) - assertEquals(0, testHarness.numProcessingTimeTimers()) - assertEquals(0, testHarness.numKeyedStateEntries()) + assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(8) + assertEquals(0, testHarness.numProcessingTimeTimers()) + assertEquals(0, testHarness.numKeyedStateEntries()) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "bbb"))) testHarness.processElement2(new StreamRecord( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala index 7f9c0ef255387..25395be75289d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala @@ -80,8 +80,8 @@ class StateCleaningCountTriggerHarnessTest { TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), GlobalWindow.get)) - // have two timers 6001 and 7002 - assertEquals(2, testHarness.numProcessingTimeTimers) + // have one timer 7002 + assertEquals(1, testHarness.numProcessingTimeTimers) assertEquals(0, testHarness.numEventTimeTimers) assertEquals(2, testHarness.numStateEntries) assertEquals(2, testHarness.numStateEntries(GlobalWindow.get)) @@ -116,9 +116,6 @@ class StateCleaningCountTriggerHarnessTest { // try to trigger onProcessingTime method via 7002, and all states are cleared val timesIt = testHarness.advanceProcessingTime(7002).iterator() - assertEquals( - TriggerResult.CONTINUE, - timesIt.next().f1) assertEquals( TriggerResult.FIRE_AND_PURGE, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala index fe90a5f3300f5..1c02889513dc7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala @@ -110,7 +110,7 @@ private class MockedKeyedProcessFunction(queryConfig: StreamQueryConfig) out: Collector[String]): Unit = { val curTime = ctx.timerService().currentProcessingTime() - registerProcessingCleanupTimer(ctx, curTime) + processCleanupTimer(ctx, curTime) state.update(value._2) } @@ -119,8 +119,12 @@ private class MockedKeyedProcessFunction(queryConfig: StreamQueryConfig) ctx: KeyedProcessFunction[String, (String, String), String]#OnTimerContext, out: Collector[String]): Unit = { - if (needToCleanupState(timestamp)) { - cleanupState(state) + if (stateCleaningEnabled) { + val cleanupTime = cleanupTimeState.value() + if (null != cleanupTime && timestamp == cleanupTime) { + // clean up + cleanupState(state) + } } } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala index 519b03f59b70e..6c0ca1a5012ef 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala @@ -110,7 +110,7 @@ private class MockedProcessFunction(queryConfig: StreamQueryConfig) out: Collector[String]): Unit = { val curTime = ctx.timerService().currentProcessingTime() - registerProcessingCleanupTimer(ctx, curTime) + processCleanupTimer(ctx, curTime) state.update(value._2) } @@ -119,7 +119,7 @@ private class MockedProcessFunction(queryConfig: StreamQueryConfig) ctx: ProcessFunction[(String, String), String]#OnTimerContext, out: Collector[String]): Unit = { - if (needToCleanupState(timestamp)) { + if (stateCleaningEnabled) { cleanupState(state) } } From 63c5d5effbb72abc8489c91dd6ead5172a455ae9 Mon Sep 17 00:00:00 2001 From: KarmaGYZ Date: Sun, 2 Dec 2018 09:49:23 +0100 Subject: [PATCH 203/359] [hotfix][docs] Add and update description of local-recovery config options This closes #7211. --- .../checkpointing_configuration.html | 4 ++-- .../configuration/CheckpointingOptions.java | 22 +++++++++++++++---- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/docs/_includes/generated/checkpointing_configuration.html b/docs/_includes/generated/checkpointing_configuration.html index 8f5ce7b341d13..6ad5eeeb09108 100644 --- a/docs/_includes/generated/checkpointing_configuration.html +++ b/docs/_includes/generated/checkpointing_configuration.html @@ -30,7 +30,7 @@
    - + @@ -50,7 +50,7 @@ - +
    state.backend.local-recovery
    falseThis option configures local recovery for this state backend. By default, local recovery is deactivated. Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend does not support local recovery and ignore this option.
    state.checkpoints.dir
    taskmanager.state.local.root-dirs
    (none)The config parameter defining the root directories for storing file-based state for local recovery. Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend does not support local recovery and ignore this option
    diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java index 6557a9f7dd401..2b4bba66c5e9e 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java @@ -71,17 +71,31 @@ public class CheckpointingOptions { /** * This option configures local recovery for this state backend. By default, local recovery is deactivated. + * + *

    Local recovery currently only covers keyed state backends. + * Currently, MemoryStateBackend does not support local recovery and ignore + * this option. */ public static final ConfigOption LOCAL_RECOVERY = ConfigOptions - .key("state.backend.local-recovery") - .defaultValue(false); + .key("state.backend.local-recovery") + .defaultValue(false) + .withDescription("This option configures local recovery for this state backend. By default, local recovery is " + + "deactivated. Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend does " + + "not support local recovery and ignore this option."); /** * The config parameter defining the root directories for storing file-based state for local recovery. + * + *

    Local recovery currently only covers keyed state backends. + * Currently, MemoryStateBackend does not support local recovery and ignore + * this option. */ public static final ConfigOption LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS = ConfigOptions - .key("taskmanager.state.local.root-dirs") - .noDefaultValue(); + .key("taskmanager.state.local.root-dirs") + .noDefaultValue() + .withDescription("The config parameter defining the root directories for storing file-based state for local " + + "recovery. Local recovery currently only covers keyed state backends. Currently, MemoryStateBackend does " + + "not support local recovery and ignore this option"); // ------------------------------------------------------------------------ // Options specific to the file-system-based state backends From f045dfd501343bc9c72c665a6c599e42a221bd67 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Tue, 11 Dec 2018 08:37:27 +0800 Subject: [PATCH 204/359] [FLINK-11123][docs] fix the import of the class is missing in ml quick start document. This closes #7269 --- docs/dev/libs/ml/quickstart.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/dev/libs/ml/quickstart.md b/docs/dev/libs/ml/quickstart.md index e056b28b5057f..2e9a7b9505c9c 100644 --- a/docs/dev/libs/ml/quickstart.md +++ b/docs/dev/libs/ml/quickstart.md @@ -153,6 +153,8 @@ A conversion can be done using a simple normalizer mapping function: {% highlight scala %} +import org.apache.flink.ml.math.Vector + def normalizer : LabeledVector => LabeledVector = { lv => LabeledVector(if (lv.label > 0.0) 1.0 else -1.0, lv.vector) } From d3966332d4fe37fd22f92c758a6a34097ed16ac6 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Tue, 11 Dec 2018 14:43:40 +0800 Subject: [PATCH 205/359] In order to maintain state compatibility revert FLINK-10543. This reverts commit 0fa9ec030d6b102f4d24f0c7f8b58c0fab97fff6. --- .../runtime/aggregate/CleanupState.scala | 57 ---- .../CoProcessFunctionWithCleanupState.scala | 69 ---- .../aggregate/GroupAggProcessFunction.scala | 4 +- ...KeyedProcessFunctionWithCleanupState.scala | 38 ++- .../aggregate/ProcTimeBoundedRangeOver.scala | 29 +- .../aggregate/ProcTimeBoundedRowsOver.scala | 4 +- .../aggregate/ProcTimeUnboundedOver.scala | 4 +- .../ProcessFunctionWithCleanupState.scala | 41 ++- .../aggregate/RowTimeBoundedRangeOver.scala | 20 +- .../aggregate/RowTimeBoundedRowsOver.scala | 8 +- .../aggregate/RowTimeUnboundedOver.scala | 8 +- .../runtime/join/NonWindowFullJoin.scala | 3 +- ...nWindowFullJoinWithNonEquiPredicates.scala | 23 +- .../runtime/join/NonWindowInnerJoin.scala | 3 +- .../table/runtime/join/NonWindowJoin.scala | 104 +++++- .../runtime/join/NonWindowLeftRightJoin.scala | 3 +- ...owLeftRightJoinWithNonEquiPredicates.scala | 25 +- .../runtime/join/NonWindowOuterJoin.scala | 9 +- ...WindowOuterJoinWithNonEquiPredicates.scala | 47 ++- .../triggers/StateCleaningCountTrigger.scala | 13 +- .../runtime/harness/JoinHarnessTest.scala | 320 ++++++++---------- ...StateCleaningCountTriggerHarnessTest.scala | 7 +- ...dProcessFunctionWithCleanupStateTest.scala | 10 +- .../ProcessFunctionWithCleanupStateTest.scala | 4 +- 24 files changed, 430 insertions(+), 423 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala deleted file mode 100644 index d9c8e2ccaee35..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CleanupState.scala +++ /dev/null @@ -1,57 +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.table.runtime.aggregate - -import org.apache.flink.api.common.state.ValueState -import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.streaming.api.functions.co.CoProcessFunction -import java.lang.{Long => JLong} - -import org.apache.flink.streaming.api.TimerService - -/** - * Base class for clean up state, both for [[ProcessFunction]] and [[CoProcessFunction]]. - */ -trait CleanupState { - - def registerProcessingCleanupTimer( - cleanupTimeState: ValueState[JLong], - currentTime: Long, - minRetentionTime: Long, - maxRetentionTime: Long, - timerService: TimerService): Unit = { - - // last registered timer - val curCleanupTime = cleanupTimeState.value() - - // check if a cleanup timer is registered and - // that the current cleanup timer won't delete state we need to keep - if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) { - // we need to register a new (later) timer - val cleanupTime = currentTime + maxRetentionTime - // register timer and remember clean-up time - timerService.registerProcessingTimeTimer(cleanupTime) - // delete expired timer - if (curCleanupTime != null) { - timerService.deleteProcessingTimeTimer(curCleanupTime) - } - cleanupTimeState.update(cleanupTime) - } - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala deleted file mode 100644 index 0c7663621ba58..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CoProcessFunctionWithCleanupState.scala +++ /dev/null @@ -1,69 +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.table.runtime.aggregate - -import java.lang.{Long => JLong} - -import org.apache.flink.api.common.state.{State, ValueState, ValueStateDescriptor} -import org.apache.flink.streaming.api.TimeDomain -import org.apache.flink.streaming.api.functions.co.CoProcessFunction -import org.apache.flink.table.api.{StreamQueryConfig, Types} - -abstract class CoProcessFunctionWithCleanupState[IN1, IN2, OUT](queryConfig: StreamQueryConfig) - extends CoProcessFunction[IN1, IN2, OUT] - with CleanupState { - - protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime - protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime - protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 - - // holds the latest registered cleanup timer - private var cleanupTimeState: ValueState[JLong] = _ - - protected def initCleanupTimeState(stateName: String) { - if (stateCleaningEnabled) { - val cleanupTimeDescriptor: ValueStateDescriptor[JLong] = - new ValueStateDescriptor[JLong](stateName, Types.LONG) - cleanupTimeState = getRuntimeContext.getState(cleanupTimeDescriptor) - } - } - - protected def processCleanupTimer( - ctx: CoProcessFunction[IN1, IN2, OUT]#Context, - currentTime: Long): Unit = { - if (stateCleaningEnabled) { - registerProcessingCleanupTimer( - cleanupTimeState, - currentTime, - minRetentionTime, - maxRetentionTime, - ctx.timerService() - ) - } - } - - protected def isProcessingTimeTimer(ctx: OnTimerContext): Boolean = { - ctx.timeDomain() == TimeDomain.PROCESSING_TIME - } - - protected def cleanupState(states: State*): Unit = { - // clear all state - states.foreach(_.clear()) - this.cleanupTimeState.clear() - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index 2d72e6de9616c..c59efe250c3d2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -86,7 +86,7 @@ class GroupAggProcessFunction( val currentTime = ctx.timerService().currentProcessingTime() // register state-cleanup timer - processCleanupTimer(ctx, currentTime) + registerProcessingCleanupTimer(ctx, currentTime) val input = inputC.row @@ -172,7 +172,7 @@ class GroupAggProcessFunction( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { cleanupState(state, cntState) function.cleanup() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala index edf5c2cd101c9..4d6840a3f43e0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/KeyedProcessFunctionWithCleanupState.scala @@ -25,15 +25,13 @@ import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFu import org.apache.flink.table.api.{StreamQueryConfig, Types} abstract class KeyedProcessFunctionWithCleanupState[K, I, O](queryConfig: StreamQueryConfig) - extends KeyedProcessFunction[K, I, O] - with CleanupState { - + extends KeyedProcessFunction[K, I, O] { protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 // holds the latest registered cleanup timer - protected var cleanupTimeState: ValueState[JLong] = _ + private var cleanupTimeState: ValueState[JLong] = _ protected def initCleanupTimeState(stateName: String) { if (stateCleaningEnabled) { @@ -43,17 +41,23 @@ abstract class KeyedProcessFunctionWithCleanupState[K, I, O](queryConfig: Stream } } - protected def processCleanupTimer( + protected def registerProcessingCleanupTimer( ctx: KeyedProcessFunction[K, I, O]#Context, currentTime: Long): Unit = { if (stateCleaningEnabled) { - registerProcessingCleanupTimer( - cleanupTimeState, - currentTime, - minRetentionTime, - maxRetentionTime, - ctx.timerService() - ) + + // last registered timer + val curCleanupTime = cleanupTimeState.value() + + // check if a cleanup timer is registered and + // that the current cleanup timer won't delete state we need to keep + if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) { + // we need to register a new (later) timer + val cleanupTime = currentTime + maxRetentionTime + // register timer and remember clean-up time + ctx.timerService().registerProcessingTimeTimer(cleanupTime) + cleanupTimeState.update(cleanupTime) + } } } @@ -61,6 +65,16 @@ abstract class KeyedProcessFunctionWithCleanupState[K, I, O](queryConfig: Stream ctx.timeDomain() == TimeDomain.PROCESSING_TIME } + protected def needToCleanupState(timestamp: Long): Boolean = { + if (stateCleaningEnabled) { + val cleanupTime = cleanupTimeState.value() + // check that the triggered timer is the last registered processing time timer. + null != cleanupTime && timestamp == cleanupTime + } else { + false + } + } + protected def cleanupState(states: State*): Unit = { // clear all state states.foreach(_.clear()) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala index 6126dc73b3085..591b942571f19 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala @@ -95,7 +95,7 @@ class ProcTimeBoundedRangeOver( val currentTime = ctx.timerService.currentProcessingTime // register state-cleanup timer - processCleanupTimer(ctx, currentTime) + registerProcessingCleanupTimer(ctx, currentTime) // buffer the event incoming event @@ -117,14 +117,11 @@ class ProcTimeBoundedRangeOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (stateCleaningEnabled) { - val cleanupTime = cleanupTimeState.value() - if (null != cleanupTime && timestamp == cleanupTime) { - // clean up and return - cleanupState(rowMapState, accumulatorState) - function.cleanup() - return - } + if (needToCleanupState(timestamp)) { + // clean up and return + cleanupState(rowMapState, accumulatorState) + function.cleanup() + return } // remove timestamp set outside of ProcessFunction. @@ -134,10 +131,11 @@ class ProcTimeBoundedRangeOver( // that have registered this time trigger 1 ms ago val currentTime = timestamp - 1 + var i = 0 // get the list of elements of current proctime val currentElements = rowMapState.get(currentTime) - // Expired clean-up timers pass the needToCleanupState check. + // Expired clean-up timers pass the needToCleanupState() check. // Perform a null check to verify that we have data to process. if (null == currentElements) { return @@ -158,6 +156,7 @@ class ProcTimeBoundedRangeOver( // and eliminate them. Multiple elements could have been received at the same timestamp // the removal of old elements happens only once per proctime as onTimer is called only once val iter = rowMapState.iterator + val markToRemove = new ArrayList[Long]() while (iter.hasNext) { val entry = iter.next() val elementKey = entry.getKey @@ -170,9 +169,17 @@ class ProcTimeBoundedRangeOver( function.retract(accumulators, retractRow) iRemove += 1 } - iter.remove() + // mark element for later removal not to modify the iterator over MapState + markToRemove.add(elementKey) } } + // need to remove in 2 steps not to have concurrent access errors via iterator to the MapState + i = 0 + while (i < markToRemove.size()) { + rowMapState.remove(markToRemove.get(i)) + i += 1 + } + // add current elements to aggregator. Multiple elements might // have arrived in the same proctime diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala index fa58ac50529ec..ccddaa5b10fc7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala @@ -110,7 +110,7 @@ class ProcTimeBoundedRowsOver( val currentTime = ctx.timerService.currentProcessingTime // register state-cleanup timer - processCleanupTimer(ctx, currentTime) + registerProcessingCleanupTimer(ctx, currentTime) // initialize state for the processed element var accumulators = accumulatorState.value @@ -187,7 +187,7 @@ class ProcTimeBoundedRowsOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { cleanupState(rowMapState, accumulatorState, counterState, smallestTsState) function.cleanup() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala index ce1a95971a9ef..6e4c510578682 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala @@ -71,7 +71,7 @@ class ProcTimeUnboundedOver( out: Collector[CRow]): Unit = { // register state-cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) val input = inputC.row @@ -95,7 +95,7 @@ class ProcTimeUnboundedOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { cleanupState(state) function.cleanup() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala index 7263de72c4b28..292fd3bdf1e27 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala @@ -26,35 +26,40 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryConfig) - extends ProcessFunction[IN, OUT] - with CleanupState { + extends ProcessFunction[IN, OUT]{ protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 // holds the latest registered cleanup timer - protected var cleanupTimeState: ValueState[JLong] = _ + private var cleanupTimeState: ValueState[JLong] = _ protected def initCleanupTimeState(stateName: String) { if (stateCleaningEnabled) { - val cleanupTimeDescriptor: ValueStateDescriptor[JLong] = + val inputCntDescriptor: ValueStateDescriptor[JLong] = new ValueStateDescriptor[JLong](stateName, Types.LONG) - cleanupTimeState = getRuntimeContext.getState(cleanupTimeDescriptor) + cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor) } } - protected def processCleanupTimer( + protected def registerProcessingCleanupTimer( ctx: ProcessFunction[IN, OUT]#Context, currentTime: Long): Unit = { if (stateCleaningEnabled) { - registerProcessingCleanupTimer( - cleanupTimeState, - currentTime, - minRetentionTime, - maxRetentionTime, - ctx.timerService() - ) + + // last registered timer + val curCleanupTime = cleanupTimeState.value() + + // check if a cleanup timer is registered and + // that the current cleanup timer won't delete state we need to keep + if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) { + // we need to register a new (later) timer + val cleanupTime = currentTime + maxRetentionTime + // register timer and remember clean-up time + ctx.timerService().registerProcessingTimeTimer(cleanupTime) + cleanupTimeState.update(cleanupTime) + } } } @@ -62,6 +67,16 @@ abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryC ctx.timeDomain() == TimeDomain.PROCESSING_TIME } + protected def needToCleanupState(timestamp: Long): Boolean = { + if (stateCleaningEnabled) { + val cleanupTime = cleanupTimeState.value() + // check that the triggered timer is the last registered processing time timer. + null != cleanupTime && timestamp == cleanupTime + } else { + false + } + } + protected def cleanupState(states: State*): Unit = { // clear all state states.foreach(_.clear()) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala index 7c509d699e8d0..b13acdf43ccab 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala @@ -114,7 +114,7 @@ class RowTimeBoundedRangeOver( val input = inputC.row // register state-cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation val triggeringTs = input.getField(rowTimeIdx).asInstanceOf[Long] @@ -143,7 +143,7 @@ class RowTimeBoundedRangeOver( out: Collector[CRow]): Unit = { if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { val keysIt = dataState.keys.iterator() val lastProcessedTime = lastTriggeringTsState.value @@ -164,7 +164,7 @@ class RowTimeBoundedRangeOver( // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. // We leave the state as it is and schedule a new cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } } return @@ -188,6 +188,9 @@ class RowTimeBoundedRangeOver( aggregatesIndex = 0 } + // keep up timestamps of retract data + val retractTsList: JList[Long] = new JArrayList[Long] + // do retraction val iter = dataState.iterator() while (iter.hasNext) { @@ -202,7 +205,7 @@ class RowTimeBoundedRangeOver( function.retract(accumulators, retractRow) dataListIndex += 1 } - iter.remove() + retractTsList.add(dataTs) } } @@ -227,13 +230,20 @@ class RowTimeBoundedRangeOver( dataListIndex += 1 } + // remove the data that has been retracted + dataListIndex = 0 + while (dataListIndex < retractTsList.size) { + dataState.remove(retractTsList.get(dataListIndex)) + dataListIndex += 1 + } + // update state accumulatorState.update(accumulators) } lastTriggeringTsState.update(timestamp) // update cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } override def close(): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index d01a499e88efc..e120d6b0afda4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala @@ -123,7 +123,7 @@ class RowTimeBoundedRowsOver( val input = inputC.row // register state-cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation val triggeringTs = input.getField(rowTimeIdx).asInstanceOf[Long] @@ -152,7 +152,7 @@ class RowTimeBoundedRowsOver( out: Collector[CRow]): Unit = { if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { val keysIt = dataState.keys.iterator() val lastProcessedTime = lastTriggeringTsState.value @@ -173,7 +173,7 @@ class RowTimeBoundedRowsOver( // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. // We leave the state as it is and schedule a new cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } } return @@ -263,7 +263,7 @@ class RowTimeBoundedRowsOver( lastTriggeringTsState.update(timestamp) // update cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } override def close(): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index 690d0d05ee304..181c7680a35a4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -108,7 +108,7 @@ abstract class RowTimeUnboundedOver( val input = inputC.row // register state-cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) val timestamp = input.getField(rowTimeIdx).asInstanceOf[Long] val curWatermark = ctx.timerService().currentWatermark() @@ -143,7 +143,7 @@ abstract class RowTimeUnboundedOver( out: Collector[CRow]): Unit = { if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { // we check whether there are still records which have not been processed yet val noRecordsToProcess = !rowMapState.keys.iterator().hasNext @@ -155,7 +155,7 @@ abstract class RowTimeUnboundedOver( // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. // We leave the state as it is and schedule a new cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } } return @@ -207,7 +207,7 @@ abstract class RowTimeUnboundedOver( } // update cleanup timer - processCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala index 5b1069ff28bc3..57c60f179c644 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala @@ -66,12 +66,13 @@ class NonWindowFullJoin( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val inputRow = value.row - updateCurrentSide(value, ctx, currentSideState) + updateCurrentSide(value, ctx, timerState, currentSideState) cRowWrapper.reset() cRowWrapper.setCollector(out) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala index 0166eef796149..9c27eb461a689 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala @@ -68,13 +68,14 @@ class NonWindowFullJoinWithNonEquiPredicates( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val currentJoinCntState = getJoinCntState(joinCntState, recordFromLeft) val inputRow = value.row - val cntAndExpiredTime = updateCurrentSide(value, ctx, currentSideState) + val cntAndExpiredTime = updateCurrentSide(value, ctx, timerState, currentSideState) if (!value.change && cntAndExpiredTime.f0 <= 0) { currentJoinCntState.remove(inputRow) } @@ -98,18 +99,18 @@ class NonWindowFullJoinWithNonEquiPredicates( } /** - * Called when a processing timer trigger. - * Expire left/right expired records and expired joinCnt state. + * Removes records which are expired from left state. Register a new timer if the state still + * holds records after the clean-up. Also, clear leftJoinCnt map state when clear left + * rowMapState. */ - override def onTimer( - timestamp: Long, - ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, - out: Collector[CRow]): Unit = { + override def expireOutTimeRow( + curTime: Long, + rowMapState: MapState[Row, JTuple2[Long, Long]], + timerState: ValueState[Long], + isLeft: Boolean, + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { - // expired timer has already been removed, delete state directly. - if (stateCleaningEnabled) { - cleanupState(leftState, rightState, joinCntState(0), joinCntState(1)) - } + expireOutTimeRow(curTime, rowMapState, timerState, isLeft, joinCntState, ctx) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala index 91a75077cd885..2e5832c269424 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala @@ -63,12 +63,13 @@ class NonWindowInnerJoin( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], isLeft: Boolean): Unit = { val inputRow = value.row - updateCurrentSide(value, ctx, currentSideState) + updateCurrentSide(value, ctx, timerState, currentSideState) cRowWrapper.setCollector(out) cRowWrapper.setChange(value.change) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala index e15cbfa550c39..c59f4c2f44b04 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.join import org.apache.flink.api.common.functions.FlatJoinFunction import org.apache.flink.api.common.functions.util.FunctionUtils -import org.apache.flink.api.common.state.{MapState, MapStateDescriptor} +import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, ValueState, ValueStateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.api.java.typeutils.TupleTypeInfo @@ -27,7 +27,6 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.co.CoProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.codegen.Compiler -import org.apache.flink.table.runtime.aggregate.CoProcessFunctionWithCleanupState import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.typeutils.TypeCheckUtils._ import org.apache.flink.table.util.Logging @@ -49,7 +48,7 @@ abstract class NonWindowJoin( genJoinFuncName: String, genJoinFuncCode: String, queryConfig: StreamQueryConfig) - extends CoProcessFunctionWithCleanupState[CRow, CRow, CRow](queryConfig) + extends CoProcessFunction[CRow, CRow, CRow] with Compiler[FlatJoinFunction[Row, Row, Row]] with Logging { @@ -63,6 +62,15 @@ abstract class NonWindowJoin( protected var rightState: MapState[Row, JTuple2[Long, Long]] = _ protected var cRowWrapper: CRowWrappingMultiOutputCollector = _ + protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime + protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime + protected val stateCleaningEnabled: Boolean = minRetentionTime > 1 + + // state to record last timer of left stream, 0 means no timer + protected var leftTimer: ValueState[Long] = _ + // state to record last timer of right stream, 0 means no timer + protected var rightTimer: ValueState[Long] = _ + // other condition function protected var joinFunction: FlatJoinFunction[Row, Row, Row] = _ @@ -70,8 +78,7 @@ abstract class NonWindowJoin( protected var curProcessTime: Long = _ override def open(parameters: Configuration): Unit = { - LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " + - s"Code:\n$genJoinFuncCode") + LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n Code:\n$genJoinFuncCode") val clazz = compile( getRuntimeContext.getUserCodeClassLoader, genJoinFuncName, @@ -93,7 +100,10 @@ abstract class NonWindowJoin( rightState = getRuntimeContext.getMapState(rightStateDescriptor) // initialize timer state - initCleanupTimeState("NonWindowJoinCleanupTime") + val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long]) + leftTimer = getRuntimeContext.getState(valueStateDescriptor1) + val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long]) + rightTimer = getRuntimeContext.getState(valueStateDescriptor2) cRowWrapper = new CRowWrappingMultiOutputCollector() LOG.debug("Instantiating NonWindowJoin.") @@ -112,7 +122,7 @@ abstract class NonWindowJoin( ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow]): Unit = { - processElement(valueC, ctx, out, leftState, rightState, isLeft = true) + processElement(valueC, ctx, out, leftTimer, leftState, rightState, isLeft = true) } /** @@ -128,7 +138,7 @@ abstract class NonWindowJoin( ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow]): Unit = { - processElement(valueC, ctx, out, rightState, leftState, isLeft = false) + processElement(valueC, ctx, out, rightTimer, rightState, leftState, isLeft = false) } /** @@ -144,13 +154,28 @@ abstract class NonWindowJoin( ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - // expired timer has already been removed, delete state directly. - if (stateCleaningEnabled) { - cleanupState(leftState, rightState) + if (stateCleaningEnabled && leftTimer.value == timestamp) { + expireOutTimeRow( + timestamp, + leftState, + leftTimer, + isLeft = true, + ctx + ) + } + + if (stateCleaningEnabled && rightTimer.value == timestamp) { + expireOutTimeRow( + timestamp, + rightState, + rightTimer, + isLeft = false, + ctx + ) } } - protected def getNewExpiredTime(curProcessTime: Long, oldExpiredTime: Long): Long = { + def getNewExpiredTime(curProcessTime: Long, oldExpiredTime: Long): Long = { if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) { curProcessTime + maxRetentionTime } else { @@ -158,15 +183,53 @@ abstract class NonWindowJoin( } } + /** + * Removes records which are expired from the state. Register a new timer if the state still + * holds records after the clean-up. + */ + def expireOutTimeRow( + curTime: Long, + rowMapState: MapState[Row, JTuple2[Long, Long]], + timerState: ValueState[Long], + isLeft: Boolean, + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { + + val rowMapIter = rowMapState.iterator() + var validTimestamp: Boolean = false + + while (rowMapIter.hasNext) { + val mapEntry = rowMapIter.next() + val recordExpiredTime = mapEntry.getValue.f1 + if (recordExpiredTime <= curTime) { + rowMapIter.remove() + } else { + // we found a timestamp that is still valid + validTimestamp = true + } + } + + // If the state has non-expired timestamps, register a new timer. + // Otherwise clean the complete state for this input. + if (validTimestamp) { + val cleanupTime = curTime + maxRetentionTime + ctx.timerService.registerProcessingTimeTimer(cleanupTime) + timerState.update(cleanupTime) + } else { + timerState.clear() + rowMapState.clear() + } + } + /** * Puts or Retract an element from the input stream into state and search the other state to * output records meet the condition. Records will be expired in state if state retention time * has been specified. */ - protected def processElement( + def processElement( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], isLeft: Boolean): Unit @@ -177,12 +240,14 @@ abstract class NonWindowJoin( * * @param value The input CRow * @param ctx The ctx to register timer or get current time + * @param timerState The state to record last timer * @param currentSideState The state to hold current side stream element * @return The row number and expired time for current input row */ - protected def updateCurrentSide( + def updateCurrentSide( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]]): JTuple2[Long, Long] = { val inputRow = value.row @@ -196,7 +261,10 @@ abstract class NonWindowJoin( cntAndExpiredTime.f1 = getNewExpiredTime(curProcessTime, cntAndExpiredTime.f1) // update timer if necessary - processCleanupTimer(ctx, curProcessTime) + if (stateCleaningEnabled && timerState.value() == 0) { + timerState.update(cntAndExpiredTime.f1) + ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime.f1) + } // update current side stream state if (!value.change) { @@ -214,7 +282,7 @@ abstract class NonWindowJoin( cntAndExpiredTime } - protected def callJoinFunction( + def callJoinFunction( inputRow: Row, inputRowFromLeft: Boolean, otherSideRow: Row, @@ -226,4 +294,8 @@ abstract class NonWindowJoin( joinFunction.join(otherSideRow, inputRow, cRowWrapper) } } + + override def close(): Unit = { + FunctionUtils.closeFunction(joinFunction) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala index 5995fb81a2c34..b4f14e494f7d5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala @@ -69,12 +69,13 @@ class NonWindowLeftRightJoin( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val inputRow = value.row - updateCurrentSide(value, ctx, currentSideState) + updateCurrentSide(value, ctx, timerState, currentSideState) cRowWrapper.reset() cRowWrapper.setCollector(out) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala index a3e25f9489ddb..33517cca5f91f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala @@ -71,13 +71,14 @@ class NonWindowLeftRightJoinWithNonEquiPredicates( value: CRow, ctx: CoProcessFunction[CRow, CRow, CRow]#Context, out: Collector[CRow], + timerState: ValueState[Long], currentSideState: MapState[Row, JTuple2[Long, Long]], otherSideState: MapState[Row, JTuple2[Long, Long]], recordFromLeft: Boolean): Unit = { val currentJoinCntState = getJoinCntState(joinCntState, recordFromLeft) val inputRow = value.row - val cntAndExpiredTime = updateCurrentSide(value, ctx, currentSideState) + val cntAndExpiredTime = updateCurrentSide(value, ctx, timerState, currentSideState) if (!value.change && cntAndExpiredTime.f0 <= 0 && recordFromLeft == isLeftJoin) { currentJoinCntState.remove(inputRow) } @@ -100,21 +101,17 @@ class NonWindowLeftRightJoinWithNonEquiPredicates( } /** - * Called when a processing timer trigger. - * Expire left/right expired records and expired joinCnt state. + * Removes records which are expired from state. Register a new timer if the state still + * holds records after the clean-up. Also, clear joinCnt map state when clear rowMapState. */ - override def onTimer( - timestamp: Long, - ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, - out: Collector[CRow]): Unit = { + override def expireOutTimeRow( + curTime: Long, + rowMapState: MapState[Row, JTuple2[Long, Long]], + timerState: ValueState[Long], + isLeft: Boolean, + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { - // expired timer has already been removed, delete state directly. - if (stateCleaningEnabled) { - cleanupState( - leftState, - rightState, - getJoinCntState(joinCntState, isLeftJoin)) - } + expireOutTimeRow(curTime, rowMapState, timerState, isLeft, joinCntState, ctx) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala index 9fadbb0eb6441..0018a16bc23ac 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala @@ -73,7 +73,7 @@ abstract class NonWindowOuterJoin( * @param otherSideState the other side state * @return the number of matched rows */ - protected def preservedJoin( + def preservedJoin( inputRow: Row, inputRowFromLeft: Boolean, otherSideState: MapState[Row, JTuple2[Long, Long]]): Long = { @@ -106,7 +106,7 @@ abstract class NonWindowOuterJoin( * RowWrapper has been reset before we call retractJoin and we also assume that the current * change of cRowWrapper is equal to value.change. */ - protected def retractJoin( + def retractJoin( value: CRow, inputRowFromLeft: Boolean, currentSideState: MapState[Row, JTuple2[Long, Long]], @@ -152,8 +152,7 @@ abstract class NonWindowOuterJoin( * Return approximate number of records in corresponding state. Only check if record number is * 0, 1 or bigger. */ - protected def approxiRecordNumInState( - currentSideState: MapState[Row, JTuple2[Long, Long]]): Long = { + def approxiRecordNumInState(currentSideState: MapState[Row, JTuple2[Long, Long]]): Long = { var recordNum = 0L val it = currentSideState.iterator() while(it.hasNext && recordNum < 2) { @@ -165,7 +164,7 @@ abstract class NonWindowOuterJoin( /** * Append input row with default null value if there is no match and Collect. */ - protected def collectAppendNull( + def collectAppendNull( inputRow: Row, inputFromLeft: Boolean, out: Collector[Row]): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala index 0eb6a8114fd76..8fe2f4fe41082 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala @@ -21,6 +21,7 @@ import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.co.CoProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.runtime.types.CRow import org.apache.flink.types.Row @@ -82,7 +83,7 @@ import org.apache.flink.types.Row * unmatched or vice versa. The RowWrapper has been reset before we call retractJoin and we * also assume that the current change of cRowWrapper is equal to value.change. */ - protected def retractJoinWithNonEquiPreds( + def retractJoinWithNonEquiPreds( value: CRow, inputRowFromLeft: Boolean, otherSideState: MapState[Row, JTuple2[Long, Long]], @@ -130,6 +131,48 @@ import org.apache.flink.types.Row } } + /** + * Removes records which are expired from state. Registers a new timer if the state still + * holds records after the clean-up. Also, clear joinCnt map state when clear rowMapState. + */ + def expireOutTimeRow( + curTime: Long, + rowMapState: MapState[Row, JTuple2[Long, Long]], + timerState: ValueState[Long], + isLeft: Boolean, + joinCntState: Array[MapState[Row, Long]], + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = { + + val currentJoinCntState = getJoinCntState(joinCntState, isLeft) + val rowMapIter = rowMapState.iterator() + var validTimestamp: Boolean = false + + while (rowMapIter.hasNext) { + val mapEntry = rowMapIter.next() + val recordExpiredTime = mapEntry.getValue.f1 + if (recordExpiredTime <= curTime) { + rowMapIter.remove() + currentJoinCntState.remove(mapEntry.getKey) + } else { + // we found a timestamp that is still valid + validTimestamp = true + } + } + // If the state has non-expired timestamps, register a new timer. + // Otherwise clean the complete state for this input. + if (validTimestamp) { + val cleanupTime = curTime + maxRetentionTime + ctx.timerService.registerProcessingTimeTimer(cleanupTime) + timerState.update(cleanupTime) + } else { + timerState.clear() + rowMapState.clear() + if (isLeft == isLeftJoin) { + currentJoinCntState.clear() + } + } + } + /** * Get left or right join cnt state. * @@ -138,7 +181,7 @@ import org.apache.flink.types.Row * @param isLeftCntState the flag whether get the left join cnt state * @return the corresponding join cnt state */ - protected def getJoinCntState( + def getJoinCntState( joinCntState: Array[MapState[Row, Long]], isLeftCntState: Boolean) : MapState[Row, Long] = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala index 6ae5e6340a635..3c18449e95321 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/triggers/StateCleaningCountTrigger.scala @@ -71,10 +71,6 @@ class StateCleaningCountTrigger(queryConfig: StreamQueryConfig, maxCount: Long) val cleanupTime = currentTime + maxRetentionTime // register timer and remember clean-up time ctx.registerProcessingTimeTimer(cleanupTime) - // delete expired timer - if (curCleanupTime != null) { - ctx.deleteProcessingTimeTimer(curCleanupTime) - } ctx.getPartitionedState(cleanupStateDesc).update(cleanupTime) } @@ -97,9 +93,12 @@ class StateCleaningCountTrigger(queryConfig: StreamQueryConfig, maxCount: Long) ctx: TriggerContext): TriggerResult = { if (stateCleaningEnabled) { - // do clear directly, since we have already deleted expired timer - clear(window, ctx) - return TriggerResult.FIRE_AND_PURGE + val cleanupTime = ctx.getPartitionedState(cleanupStateDesc).value() + // check that the triggered timer is the last registered processing time timer. + if (null != cleanupTime && time == cleanupTime) { + clear(window, ctx) + return TriggerResult.FIRE_AND_PURGE + } } TriggerResult.CONTINUE } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala index 4619c759c31af..bd19be837574c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala @@ -21,18 +21,21 @@ import java.lang.{Integer => JInt, Long => JLong} import java.util.concurrent.ConcurrentLinkedQueue import org.apache.flink.api.common.time.Time -import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.operators.join.JoinType +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness -import org.apache.flink.table.api.Types -import org.apache.flink.table.runtime.harness.HarnessTestBase.{TestStreamQueryConfig, TupleRowKeySelector} +import org.apache.flink.table.api.{StreamQueryConfig, Types} +import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, RowResultSortComparatorWithWatermarks, TestStreamQueryConfig, TupleRowKeySelector} import org.apache.flink.table.runtime.join._ import org.apache.flink.table.runtime.operators.KeyedCoProcessOperatorWithWatermarkDelay -import org.apache.flink.table.runtime.types.CRow -import org.junit.Assert.assertEquals +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.types.Row +import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Test /** @@ -827,6 +830,14 @@ class JoinHarnessTest extends HarnessTestBase { @Test def testNonWindowInnerJoin() { + val joinReturnType = CRowTypeInfo(new RowTypeInfo( + Array[TypeInformation[_]]( + INT_TYPE_INFO, + STRING_TYPE_INFO, + INT_TYPE_INFO, + STRING_TYPE_INFO), + Array("a", "b", "c", "d"))) + val joinProcessFunc = new NonWindowInnerJoin( rowType, rowType, @@ -868,32 +879,35 @@ class JoinHarnessTest extends HarnessTestBase { // right stream input and output normally testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi1"))) - // lkeys(1, 2) rkeys(1) timer_key_time(1:5, 2:6) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(6, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello1"))) - // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) - assertEquals(6, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(8, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) - // expired stream record with key value of 1 + // expired left stream record with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi2"))) - // lkeys(2) rkeys(1, 2) timer_key_time(1:9, 2:6) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(6, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) - // expired all left stream records + // expired all left stream record testHarness.setProcessingTime(6) - // lkeys() rkeys(1) timer_key_time(1:9) + assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) + + // expired right stream record with key value of 2 + testHarness.setProcessingTime(8) assertEquals(2, testHarness.numKeyedStateEntries()) assertEquals(1, testHarness.numProcessingTimeTimers()) - // expired all stream records - testHarness.setProcessingTime(9) + testHarness.setProcessingTime(10) + assertTrue(testHarness.numKeyedStateEntries() > 0) + // expired all right stream record + testHarness.setProcessingTime(11) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -961,37 +975,32 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // lkeys(1, 2) rkeys() timer_key_time(1:5, 2:6) - assertEquals(4, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello1"))) - // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "aaa"))) - // expired stream records with key value of 1 + // expired left stream record with key value of 1 testHarness.setProcessingTime(5) - // lkeys(2) rkeys(2) timer_key_time(2:6) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi2"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi2"))) - // lkeys(2) rkeys(2) timer_key_time(1:9, 2:6) - assertEquals(4, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) - // expired all stream records + // expired all left stream record testHarness.setProcessingTime(6) - // lkeys() rkeys() timer_key_time(1:9) - assertEquals(1, testHarness.numKeyedStateEntries()) - assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(3, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all data - testHarness.setProcessingTime(9) + // expired right stream record with key value of 2 + testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1058,36 +1067,32 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // lkeys(1, 2) rkeys() timer_key_time(1:5, 2:6) - assertEquals(4, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello1"))) - // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "aaa"))) - // expired stream records with key value of 1 + // expired left stream record with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi2"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi2"))) - // lkeys(2) rkeys(2) timer_key_time(1:9, 2:6) - assertEquals(4, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) - // expired stream records with key value of 2 + // expired all left stream record testHarness.setProcessingTime(6) - // lkeys() rkeys() timer_key_time(1:9) - assertEquals(1, testHarness.numKeyedStateEntries()) - assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(3, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all data - testHarness.setProcessingTime(9) + // expired right stream record with key value of 2 + testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1155,7 +1160,7 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement1(new StreamRecord( CRow(1: JInt, "bbb"))) assertEquals(1, testHarness.numProcessingTimeTimers()) - // lkeys(1) rkeys() timer_key_time(1:5) + // 1 left timer(5), 1 left key(1), 1 join cnt assertEquals(3, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) testHarness.processElement1(new StreamRecord( @@ -1163,8 +1168,7 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement1(new StreamRecord( CRow(2: JInt, "bbb"))) assertEquals(2, testHarness.numProcessingTimeTimers()) - // lkeys(1, 2) rkeys() timer_key_time(1:5, 2:6) - // l_join_cnt_keys(1, 2) + // 2 left timer(5,6), 2 left key(1,2), 2 join cnt assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1173,19 +1177,17 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "bbb"))) - // lkeys(1, 2) rkeys(1) timer_key_time(1:5, 2:6) - // l_join_cnt_keys(1, 2) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,6), 2 left keys(1,2), 2 join cnt, 1 right timer(7), 1 right key(1) + assertEquals(8, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "ccc"))) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "Hello"))) - // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) - // l_join_cnt_keys(1, 2) - assertEquals(8, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,6), 2 left keys(1,2), 2 join cnt, 2 right timer(7,8), 2 right key(1,2) + assertEquals(10, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "aaa"))) @@ -1195,29 +1197,22 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "Hi2"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // lkeys(1, 2) rkeys(2) timer_key_time(1:8, 2:6) - // l_join_cnt_keys(1, 2) - assertEquals(7, testHarness.numKeyedStateEntries()) + // expired left stream record with key value of 1 testHarness.setProcessingTime(5) - // [1]. this will clean up left stream records with expired time of 5 testHarness.processElement2(new StreamRecord( CRow(1: JInt, "Hi3"))) - // [2]. there are no elements can be connected, since be cleaned by [1] testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "Hi3"))) - // lkeys(1, 2) rkeys(2) timer_key_time(1:8, 2:6) - // l_join_cnt_keys(1, 2) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // 1 left timer(6), 1 left keys(2), 1 join cnt, 2 right timer(7,8), 1 right key(2) + assertEquals(6, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) - // expired all records with key value of 2 + // expired all left stream record testHarness.setProcessingTime(6) - // lkeys(1) rkeys() timer_key_time(1:8) - // l_join_cnt_keys(1) assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all data + // expired right stream record with key value of 2 testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1258,12 +1253,6 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "bbb", 1: JInt, "Hi1"))) expectedOutput.add(new StreamRecord( CRow(1: JInt, "bbb", null: JInt, null))) - // processing time of 5 - // timer of 8, we use only one timer state now - expectedOutput.add(new StreamRecord( - CRow(false, 1: JInt, "bbb", null: JInt, null))) - expectedOutput.add(new StreamRecord( - CRow(1: JInt, "bbb", 1: JInt, "Hi3"))) verify(expectedOutput, result) testHarness.close() @@ -1316,36 +1305,32 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // lkeys() rkeys(1, 2) timer_key_time(1:5, 2:6) - assertEquals(4, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "Hello1"))) - // lkeys(2) rkeys(1, 2) timer_key_time(1:5, 2:6) - assertEquals(5, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(7, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "aaa"))) - // expired stream records with key value of 1 + // expired right stream record with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "Hi2"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi2"))) - // lkeys(2) rkeys(2) timer_key_time(1:9, 2:6) - assertEquals(4, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) - // expired stream records with key value of 2 + // expired all right stream record testHarness.setProcessingTime(6) - // lkeys() rkeys() timer_key_time(1:9) - assertEquals(1, testHarness.numKeyedStateEntries()) - assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(3, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all data - testHarness.setProcessingTime(9) + // expired left stream record with key value of 2 + testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1413,17 +1398,15 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement2(new StreamRecord( CRow(1: JInt, "bbb"))) assertEquals(1, testHarness.numProcessingTimeTimers()) - // lkeys() rkeys(1) timer_key_time(1:5) - // r_join_cnt_keys(1) + // 1 right timer(5), 1 right key(1), 1 join cnt assertEquals(3, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "bbb"))) - // lkeys() rkeys(1, 2) timer_key_time(1:5, 2:6) - // r_join_cnt_keys(1, 2) assertEquals(2, testHarness.numProcessingTimeTimers()) + // 2 right timer(5,6), 2 right key(1,2), 2 join cnt assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1432,19 +1415,17 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "Hi1"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "bbb"))) - // lkeys(1) rkeys(1, 2) timer_key_time(1:5, 2:6) - // r_join_cnt_keys(1, 2) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // 2 right timer(5,6), 2 right keys(1,2), 2 join cnt, 1 left timer(7), 1 left key(1) + assertEquals(8, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "ccc"))) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "Hello"))) - // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) - // r_join_cnt_keys(1, 2) - assertEquals(8, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // 2 right timer(5,6), 2 right keys(1,2), 2 join cnt, 2 left timer(7,8), 2 left key(1,2) + assertEquals(10, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "aaa"))) @@ -1454,27 +1435,22 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "Hi2"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi1"))) - // lkeys(2) rkeys(1, 2) timer_key_time(1:8, 2:6) - // r_join_cnt_keys(1, 2) - assertEquals(7, testHarness.numKeyedStateEntries()) + // expired right stream record with key value of 1 testHarness.setProcessingTime(5) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "Hi3"))) testHarness.processElement1(new StreamRecord( CRow(false, 1: JInt, "Hi3"))) - // lkeys(2) rkeys(1, 2) timer_key_time(1:8, 2:6) - // r_join_cnt_keys(1, 2) - assertEquals(7, testHarness.numKeyedStateEntries()) - assertEquals(2, testHarness.numProcessingTimeTimers()) + // 1 right timer(6), 1 right keys(2), 1 join cnt, 2 left timer(7,8), 1 left key(2) + assertEquals(6, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) - // expired all stream records with key value of 2 - // lkeys() rkeys(1) timer_key_time(1:8) - // r_join_cnt_keys(1) + // expired all right stream record testHarness.setProcessingTime(6) assertEquals(3, testHarness.numKeyedStateEntries()) - assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(2, testHarness.numProcessingTimeTimers()) - // expired all data + // expired left stream record with key value of 2 testHarness.setProcessingTime(8) assertEquals(0, testHarness.numKeyedStateEntries()) assertEquals(0, testHarness.numProcessingTimeTimers()) @@ -1515,12 +1491,6 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "Hi1", 1: JInt, "bbb"))) expectedOutput.add(new StreamRecord( CRow(null: JInt, null, 1: JInt, "bbb"))) - // processing time of 5 - // timer of 8, we use only one timer state now - expectedOutput.add(new StreamRecord( - CRow(false, null: JInt, null, 1: JInt, "bbb"))) - expectedOutput.add(new StreamRecord( - CRow(1: JInt, "Hi3", 1: JInt, "bbb"))) verify(expectedOutput, result) testHarness.close() @@ -1554,8 +1524,8 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "bbb"))) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "ccc"))) - // lkeys(1) rkeys() timer_key_time(1:5) assertEquals(1, testHarness.numProcessingTimeTimers()) + // 1 left timer(5), 1 left key(1) assertEquals(2, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) @@ -1564,7 +1534,8 @@ class JoinHarnessTest extends HarnessTestBase { testHarness.processElement2(new StreamRecord( CRow(2: JInt, "ccc"))) assertEquals(2, testHarness.numProcessingTimeTimers()) - // lkeys(1) rkeys(2) timer_key_time(1:5, 2:6) + // 1 left timer(5), 1 left key(1) + // 1 right timer(6), 1 right key(1) assertEquals(4, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1572,16 +1543,18 @@ class JoinHarnessTest extends HarnessTestBase { CRow(2: JInt, "aaa"))) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "ddd"))) - // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,7), 2 left key(1,2) + // 1 right timer(6), 1 right key(1) + assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "ddd"))) - // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(6, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,7), 2 left key(1,2) + // 2 right timer(6,7), 2 right key(1,2) + assertEquals(8, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( @@ -1592,26 +1565,28 @@ class JoinHarnessTest extends HarnessTestBase { CRow(false, 1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "ddd"))) - // lkeys(1) rkeys(2) timer_key_time(1:8, 2:6) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(4, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,7), 1 left key(1) + // 2 right timer(6,7), 1 right key(2) + assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(5) - assertEquals(2, testHarness.numProcessingTimeTimers()) + assertEquals(3, testHarness.numProcessingTimeTimers()) + // 1 left timer(7) + // 2 right timer(6,7), 1 right key(2) assertEquals(4, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(6) - // lkeys(1) rkeys() timer_key_time(1:8) - assertEquals(1, testHarness.numProcessingTimeTimers()) + assertEquals(2, testHarness.numProcessingTimeTimers()) + // 1 left timer(7) + // 2 right timer(7) assertEquals(2, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(7) - assertEquals(1, testHarness.numProcessingTimeTimers()) - assertEquals(2, testHarness.numKeyedStateEntries()) - - testHarness.setProcessingTime(8) assertEquals(0, testHarness.numProcessingTimeTimers()) assertEquals(0, testHarness.numKeyedStateEntries()) + + testHarness.setProcessingTime(8) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "bbb"))) testHarness.processElement2(new StreamRecord( @@ -1718,9 +1693,8 @@ class JoinHarnessTest extends HarnessTestBase { CRow(1: JInt, "bbb"))) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "ccc"))) - // lkeys(1) rkeys() timer_key_time(1:5) - // l_join_cnt_keys(1) r_join_cnt_keys() assertEquals(1, testHarness.numProcessingTimeTimers()) + // 1 left timer(5), 1 left key(1), 1 left joincnt key(1) assertEquals(3, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(2) @@ -1728,9 +1702,9 @@ class JoinHarnessTest extends HarnessTestBase { CRow(2: JInt, "bbb"))) testHarness.processElement2(new StreamRecord( CRow(2: JInt, "ccc"))) - // lkeys(1) rkeys(2) timer_key_time(1:5, 2:6) - // l_join_cnt_keys(1) r_join_cnt_keys(2) assertEquals(2, testHarness.numProcessingTimeTimers()) + // 1 left timer(5), 1 left key(1), 1 left joincnt key(1) + // 1 right timer(6), 1 right key(1), 1 right joincnt key(1) assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(3) @@ -1738,46 +1712,46 @@ class JoinHarnessTest extends HarnessTestBase { CRow(2: JInt, "aaa"))) testHarness.processElement1(new StreamRecord( CRow(2: JInt, "ddd"))) - // lkeys(1, 2) rkeys(2) timer_key_time(1:5, 2:6) - // l_join_cnt_keys(1, 2) r_join_cnt_keys(2) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(8, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2) + // 1 right timer(6), 1 right key(1), 1 right joincnt key(1) + assertEquals(9, testHarness.numKeyedStateEntries()) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(1: JInt, "ddd"))) - // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:5, 2:6) - // l_join_cnt_keys(1, 2) r_join_cnt_keys(1, 2) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(10, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2) + // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2) + assertEquals(12, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(4) testHarness.processElement1(new StreamRecord( CRow(false, 2: JInt, "aaa"))) testHarness.processElement2(new StreamRecord( CRow(false, 1: JInt, "ddd"))) - // lkeys(1, 2) rkeys(1, 2) timer_key_time(1:8, 2:6) - // l_join_cnt_keys(1, 2) r_join_cnt_keys(1, 2) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(10, testHarness.numKeyedStateEntries()) + assertEquals(4, testHarness.numProcessingTimeTimers()) + // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2) + // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2) + assertEquals(12, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(5) - assertEquals(2, testHarness.numProcessingTimeTimers()) - assertEquals(10, testHarness.numKeyedStateEntries()) + assertEquals(3, testHarness.numProcessingTimeTimers()) + // 1 left timer(7), 1 left key(2), 1 left joincnt key(2) + // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2) + assertEquals(9, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(6) - // lkeys(1) rkeys(1) timer_key_time(1:8) - // l_join_cnt_keys(1) r_join_cnt_keys(1) - assertEquals(1, testHarness.numProcessingTimeTimers()) - assertEquals(5, testHarness.numKeyedStateEntries()) + assertEquals(2, testHarness.numProcessingTimeTimers()) + // 1 left timer(7), 1 left key(2), 1 left joincnt key(2) + // 1 right timer(7), 1 right key(2), 1 right joincnt key(2) + assertEquals(6, testHarness.numKeyedStateEntries()) testHarness.setProcessingTime(7) - assertEquals(1, testHarness.numProcessingTimeTimers()) - assertEquals(5, testHarness.numKeyedStateEntries()) - - testHarness.setProcessingTime(8) assertEquals(0, testHarness.numProcessingTimeTimers()) assertEquals(0, testHarness.numKeyedStateEntries()) + + testHarness.setProcessingTime(8) testHarness.processElement1(new StreamRecord( CRow(1: JInt, "bbb"))) testHarness.processElement2(new StreamRecord( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala index 25395be75289d..7f9c0ef255387 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/StateCleaningCountTriggerHarnessTest.scala @@ -80,8 +80,8 @@ class StateCleaningCountTriggerHarnessTest { TriggerResult.CONTINUE, testHarness.processElement(new StreamRecord(1), GlobalWindow.get)) - // have one timer 7002 - assertEquals(1, testHarness.numProcessingTimeTimers) + // have two timers 6001 and 7002 + assertEquals(2, testHarness.numProcessingTimeTimers) assertEquals(0, testHarness.numEventTimeTimers) assertEquals(2, testHarness.numStateEntries) assertEquals(2, testHarness.numStateEntries(GlobalWindow.get)) @@ -116,6 +116,9 @@ class StateCleaningCountTriggerHarnessTest { // try to trigger onProcessingTime method via 7002, and all states are cleared val timesIt = testHarness.advanceProcessingTime(7002).iterator() + assertEquals( + TriggerResult.CONTINUE, + timesIt.next().f1) assertEquals( TriggerResult.FIRE_AND_PURGE, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala index 1c02889513dc7..fe90a5f3300f5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessFunctionWithCleanupStateTest.scala @@ -110,7 +110,7 @@ private class MockedKeyedProcessFunction(queryConfig: StreamQueryConfig) out: Collector[String]): Unit = { val curTime = ctx.timerService().currentProcessingTime() - processCleanupTimer(ctx, curTime) + registerProcessingCleanupTimer(ctx, curTime) state.update(value._2) } @@ -119,12 +119,8 @@ private class MockedKeyedProcessFunction(queryConfig: StreamQueryConfig) ctx: KeyedProcessFunction[String, (String, String), String]#OnTimerContext, out: Collector[String]): Unit = { - if (stateCleaningEnabled) { - val cleanupTime = cleanupTimeState.value() - if (null != cleanupTime && timestamp == cleanupTime) { - // clean up - cleanupState(state) - } + if (needToCleanupState(timestamp)) { + cleanupState(state) } } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala index 6c0ca1a5012ef..519b03f59b70e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/ProcessFunctionWithCleanupStateTest.scala @@ -110,7 +110,7 @@ private class MockedProcessFunction(queryConfig: StreamQueryConfig) out: Collector[String]): Unit = { val curTime = ctx.timerService().currentProcessingTime() - processCleanupTimer(ctx, curTime) + registerProcessingCleanupTimer(ctx, curTime) state.update(value._2) } @@ -119,7 +119,7 @@ private class MockedProcessFunction(queryConfig: StreamQueryConfig) ctx: ProcessFunction[(String, String), String]#OnTimerContext, out: Collector[String]): Unit = { - if (stateCleaningEnabled) { + if (needToCleanupState(timestamp)) { cleanupState(state) } } From ae2c29e234ea19387f88d070739168e553bcd400 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 10 Dec 2018 16:52:02 +0100 Subject: [PATCH 206/359] [FLINK-11041][test] ReinterpretDataStreamAsKeyedStreamITCase source should hold checkpointing lock --- .../datastream/ReinterpretDataStreamAsKeyedStreamITCase.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java index 6a1b9ee75f7ef..ff28dffff1d48 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java @@ -122,7 +122,9 @@ public RandomTupleSource(int numEvents, int numKeys) { public void run(SourceContext> out) throws Exception { Random random = new Random(42); while (--remainingEvents >= 0) { - out.collect(new Tuple2<>(random.nextInt(numKeys), 1)); + synchronized (out.getCheckpointLock()) { + out.collect(new Tuple2<>(random.nextInt(numKeys), 1)); + } } } From 66d18bff76c1ec28f9ba7c112b2b3f7820cb1a2d Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 7 Dec 2018 20:42:52 +0800 Subject: [PATCH 207/359] [hotfix] [state backend, tests] Certain StateBackendMigrationTestBase tests should fail if exception isn't thrown This commit strengthens tests in StateBackendMigrationTestBase that depend on a certain state operation (restoring state, accessing state, etc.) to be failing to assert correct behaviour. However, we previously do not really fail the test if no exception was thrown when there should be. This also caught some bugs in the test itself which had the tests verifying incorrect behaviour. --- .../state/StateBackendMigrationTestBase.java | 27 ++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java index f5f30d5037a4b..dff49f2c573cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java @@ -245,6 +245,8 @@ public void testKeyedValueStateRegistrationFailsIfNewStateSerializerIsIncompatib // the new serializer is INCOMPATIBLE, so registering the state should fail backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); + + Assert.fail("should have failed"); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); }finally { @@ -288,10 +290,12 @@ public void testKeyedListStateRegistrationFailsIfNewStateSerializerIsIncompatibl kvId = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); // the new serializer is INCOMPATIBLE, so registering the state should fail backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); + + Assert.fail("should have failed"); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } finally { @@ -358,6 +362,8 @@ public void testStateBackendCreationFailsIfNewKeySerializerIsNotCompatible() thr try { // the new key serializer is incompatible; this should fail the restore restoreKeyedBackend(new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE), snapshot); + + Assert.fail("should have failed"); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } @@ -365,6 +371,8 @@ public void testStateBackendCreationFailsIfNewKeySerializerIsNotCompatible() thr try { // the new key serializer requires migration; this should fail the restore restoreKeyedBackend(new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION), snapshot); + + Assert.fail("should have failed"); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } @@ -397,26 +405,33 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsNotCompatib KeyedStateHandle snapshot = runSnapshot( backend.snapshot(1L, 2L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()), sharedStateRegistry); - backend.dispose(); + // test incompatible namespace serializer; start with a freshly restored backend + backend.dispose(); backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot); - try { // the new namespace serializer is incompatible; this should fail the restore backend.getPartitionedState( new TestType("namespace", 123), new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE), kvId); + + Assert.fail("should have failed"); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } + // test namespace serializer that requires migration; start with a freshly restored backend + backend.dispose(); + backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot); try { // the new namespace serializer requires migration; this should fail the restore backend.getPartitionedState( new TestType("namespace", 123), new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION), kvId); + + Assert.fail("should have failed"); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } @@ -685,10 +700,12 @@ public void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatibl descriptor = new MapStateDescriptor<>( stateName, IntSerializer.INSTANCE, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); // the new value serializer is INCOMPATIBLE, so registering the state should fail backend.getBroadcastState(descriptor); + + Assert.fail("should have failed."); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } finally { @@ -726,6 +743,8 @@ public void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible( // the new key serializer is INCOMPATIBLE, so registering the state should fail backend.getBroadcastState(descriptor); + + Assert.fail("should have failed."); } catch (Exception e) { Assert.assertTrue(ExceptionUtils.findThrowable(e, StateMigrationException.class).isPresent()); } finally { From 4a331186be6775bf2948426655b5b61e0b7eeda2 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 7 Dec 2018 20:47:30 +0800 Subject: [PATCH 208/359] [hotfix] [state backends] New namespace serializer in HeapKeyedStateBackend should always be compatible Previously, we were only checking if the new namespace serializer is incompatible, while properly we should be checking that it is strictly compatible. This doesn't affect any user expected behaviour, since the namespace serializer is never exposed to users. --- .../flink/runtime/state/heap/HeapKeyedStateBackend.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java index 4eff3a285bbdc..ecad76c798f37 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java @@ -283,8 +283,8 @@ private StateTable tryRegisterStateTable( TypeSerializerSchemaCompatibility namespaceCompatibility = namespaceSerializerSnapshot.resolveSchemaCompatibility(namespaceSerializer); - if (namespaceCompatibility.isIncompatible()) { - throw new StateMigrationException("For heap backends, the new namespace serializer must not be incompatible."); + if (!namespaceCompatibility.isCompatibleAsIs()) { + throw new StateMigrationException("For heap backends, the new namespace serializer must be compatible."); } @SuppressWarnings("unchecked") From a3acb4ec0df547b7f0d4f20b3aa0eb860ec700b5 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 8 Dec 2018 18:28:13 +0800 Subject: [PATCH 209/359] [hotfix] [state backends, tests] Make StateBackendMigrationTestBase more understandable The StateBackendMigrationTestBase previously mocked user behaviour of upgrading serializers by using a single serializer class, that can be configured with different target compatibility results when they are checked for compatibility. This is a bit hard to understand, also doesn't really reflect how a user would actually approach the feature. Instead, instead of configuring a single serializer class with different compatibility "personalities", this commit uses actual different classes, V1TestTypeSerializer, V2TestTypeSerializer, and IncompatibleTestTypeSerializer, to simulate the compatibility cases in tests. This commit also refactors the serializer migration related test serializers / types / snapshots to the testutil package, so that it can be shared by other state migration related tests in the future. --- .../state/StateBackendMigrationTestBase.java | 301 +++--------------- .../testutils/statemigration/TestType.java | 237 ++++++++++++++ .../V1TestTypeSerializerSnapshot.java | 62 ++++ .../V2TestTypeSerializerSnapshot.java | 60 ++++ 4 files changed, 395 insertions(+), 265 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/TestType.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V1TestTypeSerializerSnapshot.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V2TestTypeSerializerSnapshot.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java index dff49f2c573cf..5511792673e9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java @@ -34,21 +34,20 @@ import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; -import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.apache.flink.runtime.testutils.statemigration.TestType; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.StateMigrationException; import org.apache.flink.util.TestLogger; + import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nonnull; import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Objects; import java.util.concurrent.RunnableFuture; /** @@ -69,17 +68,6 @@ public abstract class StateBackendMigrationTestBase kvId = new ValueStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ValueState valueState = backend .getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -113,10 +101,10 @@ public void testKeyedValueStateMigration() throws Exception { backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot); - // the new serializer is REQUIRES_MIGRATION, and has a completely new serialization schema. + // the new serializer is V2, and has a completely new serialization schema. kvId = new ValueStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestType.V2TestTypeSerializer()); valueState = backend .getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -151,7 +139,7 @@ public void testKeyedListStateMigration() throws Exception { try { ListStateDescriptor kvId = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ListState listState = backend .getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -174,10 +162,10 @@ public void testKeyedListStateMigration() throws Exception { backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot); - // the new serializer is REQUIRES_MIGRATION, and has a completely new serialization schema. + // the new serializer is V2, and has a completely new serialization schema. kvId = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestType.V2TestTypeSerializer()); listState = backend .getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -221,7 +209,7 @@ public void testKeyedValueStateRegistrationFailsIfNewStateSerializerIsIncompatib try { ValueStateDescriptor kvId = new ValueStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ValueState valueState = backend .getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -241,7 +229,7 @@ public void testKeyedValueStateRegistrationFailsIfNewStateSerializerIsIncompatib kvId = new ValueStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); + new TestType.IncompatibleTestTypeSerializer()); // the new serializer is INCOMPATIBLE, so registering the state should fail backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -265,7 +253,7 @@ public void testKeyedListStateRegistrationFailsIfNewStateSerializerIsIncompatibl try { ListStateDescriptor kvId = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ListState listState = backend .getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -290,7 +278,7 @@ public void testKeyedListStateRegistrationFailsIfNewStateSerializerIsIncompatibl kvId = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); + new TestType.IncompatibleTestTypeSerializer()); // the new serializer is INCOMPATIBLE, so registering the state should fail backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId); @@ -312,7 +300,7 @@ public void testPriorityQueueStateCreationFailsIfNewSerializerIsNotCompatible() try { InternalPriorityQueue internalPriorityQueue = backend.create( - "testPriorityQueue", new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + "testPriorityQueue", new TestType.V1TestTypeSerializer()); internalPriorityQueue.add(new TestType("key-1", 123)); internalPriorityQueue.add(new TestType("key-2", 346)); @@ -325,7 +313,7 @@ public void testPriorityQueueStateCreationFailsIfNewSerializerIsNotCompatible() backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot); backend.create( - "testPriorityQueue", new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); + "testPriorityQueue", new TestType.IncompatibleTestTypeSerializer()); Assert.fail("should have failed"); } catch (Exception e) { @@ -341,7 +329,7 @@ public void testStateBackendCreationFailsIfNewKeySerializerIsNotCompatible() thr SharedStateRegistry sharedStateRegistry = new SharedStateRegistry(); AbstractKeyedStateBackend backend = createKeyedBackend( - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); final String stateName = "test-name"; try { @@ -361,7 +349,7 @@ public void testStateBackendCreationFailsIfNewKeySerializerIsNotCompatible() thr try { // the new key serializer is incompatible; this should fail the restore - restoreKeyedBackend(new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE), snapshot); + restoreKeyedBackend(new TestType.IncompatibleTestTypeSerializer(), snapshot); Assert.fail("should have failed"); } catch (Exception e) { @@ -370,7 +358,7 @@ public void testStateBackendCreationFailsIfNewKeySerializerIsNotCompatible() thr try { // the new key serializer requires migration; this should fail the restore - restoreKeyedBackend(new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION), snapshot); + restoreKeyedBackend(new TestType.V2TestTypeSerializer(), snapshot); Assert.fail("should have failed"); } catch (Exception e) { @@ -394,7 +382,7 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsNotCompatib ValueState valueState = backend .getPartitionedState( new TestType("namespace", 123), - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS), + new TestType.V1TestTypeSerializer(), kvId); backend.setCurrentKey(1); @@ -413,7 +401,7 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsNotCompatib // the new namespace serializer is incompatible; this should fail the restore backend.getPartitionedState( new TestType("namespace", 123), - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE), + new TestType.IncompatibleTestTypeSerializer(), kvId); Assert.fail("should have failed"); @@ -428,7 +416,7 @@ public void testKeyedStateRegistrationFailsIfNewNamespaceSerializerIsNotCompatib // the new namespace serializer requires migration; this should fail the restore backend.getPartitionedState( new TestType("namespace", 123), - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION), + new TestType.V2TestTypeSerializer(), kvId); Assert.fail("should have failed"); @@ -454,7 +442,7 @@ public void testOperatorParitionableListStateMigration() throws Exception { try { ListStateDescriptor descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ListState state = backend.getListState(descriptor); state.add(new TestType("foo", 13)); @@ -468,7 +456,7 @@ public void testOperatorParitionableListStateMigration() throws Exception { descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestType.V2TestTypeSerializer()); state = backend.getListState(descriptor); // the state backend should have decided whether or not it needs to perform state migration; @@ -493,7 +481,7 @@ public void testUnionListStateMigration() throws Exception { try { ListStateDescriptor descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ListState state = backend.getUnionListState(descriptor); state.add(new TestType("foo", 13)); @@ -507,7 +495,7 @@ public void testUnionListStateMigration() throws Exception { descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestType.V2TestTypeSerializer()); state = backend.getUnionListState(descriptor); // the state backend should have decided whether or not it needs to perform state migration; @@ -533,7 +521,7 @@ public void testBroadcastStateValueMigration() throws Exception { MapStateDescriptor descriptor = new MapStateDescriptor<>( stateName, IntSerializer.INSTANCE, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); BroadcastState state = backend.getBroadcastState(descriptor); state.put(3, new TestType("foo", 13)); @@ -548,7 +536,7 @@ public void testBroadcastStateValueMigration() throws Exception { descriptor = new MapStateDescriptor<>( stateName, IntSerializer.INSTANCE, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION)); + new TestType.V2TestTypeSerializer()); state = backend.getBroadcastState(descriptor); // the state backend should have decided whether or not it needs to perform state migration; @@ -571,7 +559,7 @@ public void testBroadcastStateKeyMigration() throws Exception { try { MapStateDescriptor descriptor = new MapStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS), + new TestType.V1TestTypeSerializer(), IntSerializer.INSTANCE); BroadcastState state = backend.getBroadcastState(descriptor); @@ -586,7 +574,7 @@ public void testBroadcastStateKeyMigration() throws Exception { descriptor = new MapStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.REQUIRES_MIGRATION), + new TestType.V2TestTypeSerializer(), IntSerializer.INSTANCE); state = backend.getBroadcastState(descriptor); @@ -610,7 +598,7 @@ public void testOperatorParitionableListStateRegistrationFailsIfNewSerializerIsI try { ListStateDescriptor descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ListState state = backend.getListState(descriptor); state.add(new TestType("foo", 13)); @@ -624,7 +612,7 @@ public void testOperatorParitionableListStateRegistrationFailsIfNewSerializerIsI descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); + new TestType.IncompatibleTestTypeSerializer()); // the new serializer is INCOMPATIBLE, so registering the state should fail backend.getListState(descriptor); @@ -647,7 +635,7 @@ public void testUnionListStateRegistrationFailsIfNewSerializerIsIncompatible() t try { ListStateDescriptor descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); ListState state = backend.getUnionListState(descriptor); state.add(new TestType("foo", 13)); @@ -661,7 +649,7 @@ public void testUnionListStateRegistrationFailsIfNewSerializerIsIncompatible() t descriptor = new ListStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); + new TestType.IncompatibleTestTypeSerializer()); // the new serializer is INCOMPATIBLE, so registering the state should fail backend.getUnionListState(descriptor); @@ -685,7 +673,7 @@ public void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatibl MapStateDescriptor descriptor = new MapStateDescriptor<>( stateName, IntSerializer.INSTANCE, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS)); + new TestType.V1TestTypeSerializer()); BroadcastState state = backend.getBroadcastState(descriptor); state.put(3, new TestType("foo", 13)); @@ -700,7 +688,7 @@ public void testBroadcastStateRegistrationFailsIfNewValueSerializerIsIncompatibl descriptor = new MapStateDescriptor<>( stateName, IntSerializer.INSTANCE, - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE)); + new TestType.IncompatibleTestTypeSerializer()); // the new value serializer is INCOMPATIBLE, so registering the state should fail backend.getBroadcastState(descriptor); @@ -723,7 +711,7 @@ public void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible( try { MapStateDescriptor descriptor = new MapStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS), + new TestType.V1TestTypeSerializer(), IntSerializer.INSTANCE); BroadcastState state = backend.getBroadcastState(descriptor); @@ -738,7 +726,7 @@ public void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible( descriptor = new MapStateDescriptor<>( stateName, - new TestSerializer(SerializerCompatibilityType.INCOMPATIBLE), + new TestType.IncompatibleTestTypeSerializer(), IntSerializer.INSTANCE); // the new key serializer is INCOMPATIBLE, so registering the state should fail @@ -756,223 +744,6 @@ public void testBroadcastStateRegistrationFailsIfNewKeySerializerIsIncompatible( // Test types, serializers, and serializer snapshots // ------------------------------------------------------------------------------- - /** - * The type used as state under tests. - * - *

    This is implemented so that the type can also be used as keyed priority queue state. - */ - private static class TestType implements HeapPriorityQueueElement, PriorityComparable, Keyed { - - private int index; - - private final int value; - private final String key; - - public TestType(String key, int value) { - this.key = key; - this.value = value; - } - - @Override - public String getKey() { - return key; - } - - @Override - public int comparePriorityTo(@Nonnull TestType other) { - return Integer.compare(value, other.value); - } - - @Override - public int getInternalIndex() { - return index; - } - - @Override - public void setInternalIndex(int newIndex) { - this.index = newIndex; - } - - @Override - public boolean equals(Object obj) { - if (obj == null || !(obj instanceof StateBackendMigrationTestBase.TestType)) { - return false; - } - - if (obj == this) { - return true; - } - - TestType other = (TestType) obj; - return Objects.equals(key, other.key) && value == other.value; - } - - @Override - public int hashCode() { - return 31 * key.hashCode() + value; - } - } - - private static class TestSerializer extends TypeSerializer { - - private static final String MIGRATION_PAYLOAD = "random-migration-payload"; - - private final SerializerCompatibilityType compatibilityType; - - TestSerializer(SerializerCompatibilityType compatibilityType) { - this.compatibilityType = compatibilityType; - } - - // -------------------------------------------------------------------------------- - // State serialization relevant methods - // -------------------------------------------------------------------------------- - - @Override - public void serialize(TestType record, DataOutputView target) throws IOException { - switch (compatibilityType) { - case COMPATIBLE_AS_IS: - target.writeUTF(record.getKey()); - target.writeInt(record.value); - break; - - case REQUIRES_MIGRATION: - target.writeUTF(record.getKey()); - target.writeUTF(MIGRATION_PAYLOAD); - target.writeInt(record.value); - target.writeBoolean(true); - break; - - case INCOMPATIBLE: - // the serializer shouldn't be used in this case - throw new UnsupportedOperationException(); - } - } - - @Override - public TestType deserialize(DataInputView source) throws IOException { - String key; - int value; - - switch (compatibilityType) { - case COMPATIBLE_AS_IS: - key = source.readUTF(); - value = source.readInt(); - break; - - case REQUIRES_MIGRATION: - key = source.readUTF(); - Assert.assertEquals(MIGRATION_PAYLOAD, source.readUTF()); - value = source.readInt(); - Assert.assertTrue(source.readBoolean()); - break; - - default: - case INCOMPATIBLE: - // the serializer shouldn't be used in this case - throw new UnsupportedOperationException(); - } - - return new TestType(key, value); - } - - @Override - public TestType copy(TestType from) { - return new TestType(from.key, from.value); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new TestSerializerSnapshot(); - } - - // -------------------------------------------------------------------------------- - // Miscellaneous serializer methods - // -------------------------------------------------------------------------------- - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - serialize(deserialize(source), target); - } - - @Override - public TestType deserialize(TestType reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public TestType copy(TestType from, TestType reuse) { - return copy(from); - } - - @Override - public TestType createInstance() { - throw new UnsupportedOperationException(); - } - - @Override - public TypeSerializer duplicate() { - return this; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public int getLength() { - return -1; - } - - @Override - public boolean canEqual(Object obj) { - return getClass().equals(obj.getClass()); - } - - @Override - public int hashCode() { - return getClass().hashCode(); - } - - @Override - public boolean equals(Object obj) { - return obj == this; - } - } - - public static class TestSerializerSnapshot implements TypeSerializerSnapshot { - - @Override - public int getCurrentVersion() { - return 1; - } - - @Override - public TypeSerializer restoreSerializer() { - return new TestSerializer(SerializerCompatibilityType.COMPATIBLE_AS_IS); - } - - @Override - public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { - switch (((TestSerializer) newSerializer).compatibilityType) { - case COMPATIBLE_AS_IS: - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - case REQUIRES_MIGRATION: - return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); - case INCOMPATIBLE: - return TypeSerializerSchemaCompatibility.incompatible(); - default: - throw new UnsupportedOperationException(); - } - } - - @Override - public void writeSnapshot(DataOutputView out) throws IOException {} - - @Override - public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException {} - } - public static class CustomVoidNamespaceSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/TestType.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/TestType.java new file mode 100644 index 0000000000000..e3b0a066e5158 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/TestType.java @@ -0,0 +1,237 @@ +/* + * 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.testutils.statemigration; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.state.Keyed; +import org.apache.flink.runtime.state.PriorityComparable; +import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement; +import org.junit.Assert; + +import javax.annotation.Nonnull; +import java.io.IOException; +import java.util.Objects; + +/** + * A data type used as state in state migration tests. + * + *

    This is implemented so that the type can also be used as keyed priority queue state. + */ +public class TestType implements HeapPriorityQueueElement, PriorityComparable, Keyed { + + private int index; + + private final int value; + private final String key; + + public TestType(String key, int value) { + this.key = key; + this.value = value; + } + + @Override + public String getKey() { + return key; + } + + public int getValue() { + return value; + } + + @Override + public int comparePriorityTo(@Nonnull TestType other) { + return Integer.compare(value, other.value); + } + + @Override + public int getInternalIndex() { + return index; + } + + @Override + public void setInternalIndex(int newIndex) { + this.index = newIndex; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || !(obj instanceof TestType)) { + return false; + } + + if (obj == this) { + return true; + } + + TestType other = (TestType) obj; + return Objects.equals(key, other.key) && value == other.value; + } + + @Override + public int hashCode() { + return 31 * key.hashCode() + value; + } + + /** + * A serializer that read / writes {@link TestType} in schema version 1. + */ + public static class V1TestTypeSerializer extends TestTypeSerializerBase { + private static final long serialVersionUID = 5053346160938769779L; + + @Override + public void serialize(TestType record, DataOutputView target) throws IOException { + target.writeUTF(record.getKey()); + target.writeInt(record.getValue()); + } + + @Override + public TestType deserialize(DataInputView source) throws IOException { + return new TestType(source.readUTF(), source.readInt()); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new V1TestTypeSerializerSnapshot(); + } + } + + /** + * A serializer that read / writes {@link TestType} in schema version 2. + * Migration is required if the state was previously written with {@link V1TestTypeSerializer}. + */ + public static class V2TestTypeSerializer extends TestTypeSerializerBase { + + private static final long serialVersionUID = 7199590310936186578L; + + private static final String RANDOM_PAYLOAD = "random-payload"; + + @Override + public void serialize(TestType record, DataOutputView target) throws IOException { + target.writeUTF(record.getKey()); + target.writeUTF(RANDOM_PAYLOAD); + target.writeInt(record.getValue()); + target.writeBoolean(true); + } + + @Override + public TestType deserialize(DataInputView source) throws IOException { + String key = source.readUTF(); + Assert.assertEquals(RANDOM_PAYLOAD, source.readUTF()); + int value = source.readInt(); + Assert.assertTrue(source.readBoolean()); + + return new TestType(key, value); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new V1TestTypeSerializerSnapshot(); + } + } + + /** + * A serializer that is meant to be incompatible with any of the serializers. + */ + public static class IncompatibleTestTypeSerializer extends TestTypeSerializerBase { + + private static final long serialVersionUID = -2959080770523247215L; + + @Override + public void serialize(TestType record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException("This is an incompatible serializer; shouldn't be used."); + } + + @Override + public TestType deserialize(DataInputView source) throws IOException { + throw new UnsupportedOperationException("This is an incompatible serializer; shouldn't be used."); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException("This is an incompatible serializer; shouldn't be used."); + } + } + + public static abstract class TestTypeSerializerBase extends TypeSerializer { + + private static final long serialVersionUID = 256299937766275871L; + + // -------------------------------------------------------------------------------- + // Miscellaneous serializer methods + // -------------------------------------------------------------------------------- + + @Override + public TestType copy(TestType from) { + return new TestType(from.getKey(), from.getValue()); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public TestType deserialize(TestType reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public TestType copy(TestType from, TestType reuse) { + return copy(from); + } + + @Override + public TestType createInstance() { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializer duplicate() { + return this; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public boolean canEqual(Object obj) { + return getClass().equals(obj.getClass()); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public boolean equals(Object obj) { + return obj == this; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V1TestTypeSerializerSnapshot.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V1TestTypeSerializerSnapshot.java new file mode 100644 index 0000000000000..b2b802a30c2c5 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V1TestTypeSerializerSnapshot.java @@ -0,0 +1,62 @@ +/* + * 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.testutils.statemigration; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** + * Snapshot class for {@link TestType.V1TestTypeSerializer}. + */ +public class V1TestTypeSerializerSnapshot implements TypeSerializerSnapshot { + + @Override + public int getCurrentVersion() { + return 1; + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { + if (newSerializer instanceof TestType.V1TestTypeSerializer) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } else if (newSerializer instanceof TestType.V2TestTypeSerializer) { + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); + } else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + + @Override + public TypeSerializer restoreSerializer() { + return new TestType.V1TestTypeSerializer(); + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V2TestTypeSerializerSnapshot.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V2TestTypeSerializerSnapshot.java new file mode 100644 index 0000000000000..3cd4fff8d8663 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/statemigration/V2TestTypeSerializerSnapshot.java @@ -0,0 +1,60 @@ +/* + * 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.testutils.statemigration; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** + * Snapshot class for {@link TestType.V2TestTypeSerializer}. + */ +public class V2TestTypeSerializerSnapshot implements TypeSerializerSnapshot { + + @Override + public int getCurrentVersion() { + return 1; + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { + if (newSerializer instanceof TestType.V2TestTypeSerializer) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } else { + return TypeSerializerSchemaCompatibility.incompatible(); + } + } + + @Override + public TypeSerializer restoreSerializer() { + return new TestType.V2TestTypeSerializer(); + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + } +} From 167b4eaa68322f4d4b36d21d0ba62ba742d6401e Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 9 Dec 2018 15:51:59 +0800 Subject: [PATCH 210/359] [FLINK-11094] [rocksdb] Eagerly create meta infos for restored state in RocksDBKeyedStateBackend This ensures that all restored state, even non-accessed ones after the restore, have a meta info available on future snapshots. --- .../state/RegisteredStateMetaInfoBase.java | 17 +++++++++++++++++ .../state/RocksDBKeyedStateBackend.java | 18 +++++++++++++++--- 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredStateMetaInfoBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredStateMetaInfoBase.java index 4132d144a4adb..b7dff59aef036 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredStateMetaInfoBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredStateMetaInfoBase.java @@ -42,4 +42,21 @@ public String getName() { @Nonnull public abstract StateMetaInfoSnapshot snapshot(); + + public static RegisteredStateMetaInfoBase fromMetaInfoSnapshot(@Nonnull StateMetaInfoSnapshot snapshot) { + + final StateMetaInfoSnapshot.BackendStateType backendStateType = snapshot.getBackendStateType(); + switch (backendStateType) { + case KEY_VALUE: + return new RegisteredKeyValueStateBackendMetaInfo<>(snapshot); + case OPERATOR: + return new RegisteredOperatorStateBackendMetaInfo<>(snapshot); + case BROADCAST: + return new RegisteredBroadcastStateBackendMetaInfo<>(snapshot); + case PRIORITY_QUEUE: + return new RegisteredPriorityQueueStateBackendMetaInfo<>(snapshot); + default: + throw new IllegalArgumentException("Unknown backend state type: " + backendStateType); + } + } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 50caa0d912a55..d12370bd7f65d 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -758,7 +758,11 @@ private void restoreKVStateMetaData() throws IOException, StateMigrationExceptio ColumnFamilyHandle columnFamily = rocksDBKeyedStateBackend.db.createColumnFamily(columnFamilyDescriptor); - registeredColumn = new Tuple2<>(columnFamily, null); + // create a meta info for the state on restore; + // this allows us to retain the state in future snapshots even if it wasn't accessed + RegisteredStateMetaInfoBase stateMetaInfo = + RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(restoredMetaInfo); + registeredColumn = new Tuple2<>(columnFamily, stateMetaInfo); rocksDBKeyedStateBackend.kvStateInformation.put(restoredMetaInfo.getName(), registeredColumn); } else { @@ -1069,10 +1073,14 @@ private ColumnFamilyHandle getOrRegisterColumnFamilyHandle( stateBackend.kvStateInformation.get(stateMetaInfoSnapshot.getName()); if (null == registeredStateMetaInfoEntry) { + // create a meta info for the state on restore; + // this allows us to retain the state in future snapshots even if it wasn't accessed + RegisteredStateMetaInfoBase stateMetaInfo = + RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(stateMetaInfoSnapshot); registeredStateMetaInfoEntry = new Tuple2<>( columnFamilyHandle != null ? columnFamilyHandle : stateBackend.db.createColumnFamily(columnFamilyDescriptor), - null); + stateMetaInfo); stateBackend.registerKvStateInformation( stateMetaInfoSnapshot.getName(), @@ -1201,9 +1209,13 @@ private void restoreLocalStateIntoFullInstance( ColumnFamilyHandle columnFamilyHandle = columnFamilyHandles.get(i); + // create a meta info for the state on restore; + // this allows us to retain the state in future snapshots even if it wasn't accessed + RegisteredStateMetaInfoBase stateMetaInfo = + RegisteredStateMetaInfoBase.fromMetaInfoSnapshot(stateMetaInfoSnapshot); stateBackend.registerKvStateInformation( stateMetaInfoSnapshot.getName(), - new Tuple2<>(columnFamilyHandle, null)); + new Tuple2<>(columnFamilyHandle, stateMetaInfo)); } // use the restore sst files as the base for succeeding checkpoints From ba346cade85511086c08c634cc34b1742d9d98c8 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 9 Dec 2018 15:15:04 +0800 Subject: [PATCH 211/359] [FLINK-11094] [state backends] Let meta infos always lazily access restore serializer This commit introduces StateSerializerProvider that wraps logic on how to obtain serializers for registered state, either with the previous schema of state in checkpoints or the current schema of state. All state meta info subclasses use StateSerializerProviders to replace direct serializer instances. This allows meta infos that were instantiated with restored serializer snapshots to not eagerly access the restore serializer when restoring state. This needs to be avoided since when restoring from 1.6, the restore serializer might not be available; for RocksDB, this should be tolerable. --- ...gisteredBroadcastStateBackendMetaInfo.java | 84 ++++-- ...egisteredKeyValueStateBackendMetaInfo.java | 115 +++++--- ...egisteredOperatorStateBackendMetaInfo.java | 46 +++- ...eredPriorityQueueStateBackendMetaInfo.java | 39 ++- .../state/StateSerializerProvider.java | 245 ++++++++++++++++++ .../runtime/state/StateBackendTestBase.java | 51 ++++ .../state/StateSerializerProviderTest.java | 187 +++++++++++++ .../state/RocksDBKeyedStateBackend.java | 125 ++++----- 8 files changed, 739 insertions(+), 153 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSerializerProvider.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSerializerProviderTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java index 70a14142474c2..95a650ef3d33c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java @@ -19,11 +19,13 @@ package org.apache.flink.runtime.state; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.HashMap; @@ -38,11 +40,11 @@ public class RegisteredBroadcastStateBackendMetaInfo extends RegisteredSta /** The type serializer for the keys in the map state. */ @Nonnull - private final TypeSerializer keySerializer; + private final StateSerializerProvider keySerializerProvider; /** The type serializer for the values in the map state. */ @Nonnull - private final TypeSerializer valueSerializer; + private final StateSerializerProvider valueSerializerProvider; public RegisteredBroadcastStateBackendMetaInfo( @Nonnull final String name, @@ -50,19 +52,19 @@ public RegisteredBroadcastStateBackendMetaInfo( @Nonnull final TypeSerializer keySerializer, @Nonnull final TypeSerializer valueSerializer) { - super(name); - Preconditions.checkArgument(assignmentMode == OperatorStateHandle.Mode.BROADCAST); - this.assignmentMode = assignmentMode; - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; + this( + name, + assignmentMode, + StateSerializerProvider.fromNewState(keySerializer), + StateSerializerProvider.fromNewState(valueSerializer)); } public RegisteredBroadcastStateBackendMetaInfo(@Nonnull RegisteredBroadcastStateBackendMetaInfo copy) { this( Preconditions.checkNotNull(copy).name, copy.assignmentMode, - copy.keySerializer.duplicate(), - copy.valueSerializer.duplicate()); + copy.getKeySerializer().duplicate(), + copy.getValueSerializer().duplicate()); } @SuppressWarnings("unchecked") @@ -71,10 +73,13 @@ public RegisteredBroadcastStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sn snapshot.getName(), OperatorStateHandle.Mode.valueOf( snapshot.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.OPERATOR_STATE_DISTRIBUTION_MODE)), - (TypeSerializer) Preconditions.checkNotNull( - snapshot.restoreTypeSerializer(StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER)), - (TypeSerializer) Preconditions.checkNotNull( - snapshot.restoreTypeSerializer(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))); + StateSerializerProvider.fromRestoredState( + (TypeSerializerSnapshot) Preconditions.checkNotNull( + snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER))), + StateSerializerProvider.fromRestoredState( + (TypeSerializerSnapshot) Preconditions.checkNotNull( + snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)))); + Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.BROADCAST == snapshot.getBackendStateType()); } @@ -86,6 +91,19 @@ public RegisteredBroadcastStateBackendMetaInfo deepCopy() { return new RegisteredBroadcastStateBackendMetaInfo<>(this); } + private RegisteredBroadcastStateBackendMetaInfo( + @Nonnull final String name, + @Nonnull final OperatorStateHandle.Mode assignmentMode, + @Nonnull final StateSerializerProvider keySerializerProvider, + @Nonnull final StateSerializerProvider valueSerializerProvider) { + + super(name); + Preconditions.checkArgument(assignmentMode == OperatorStateHandle.Mode.BROADCAST); + this.assignmentMode = assignmentMode; + this.keySerializerProvider = keySerializerProvider; + this.valueSerializerProvider = valueSerializerProvider; + } + @Nonnull @Override public StateMetaInfoSnapshot snapshot() { @@ -94,12 +112,32 @@ public StateMetaInfoSnapshot snapshot() { @Nonnull public TypeSerializer getKeySerializer() { - return keySerializer; + return keySerializerProvider.currentSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility updateKeySerializer(TypeSerializer newKeySerializer) { + return keySerializerProvider.registerNewSerializerForRestoredState(newKeySerializer); + } + + @Nullable + public TypeSerializer getPreviousKeySerializer() { + return keySerializerProvider.previousSchemaSerializer(); } @Nonnull public TypeSerializer getValueSerializer() { - return valueSerializer; + return valueSerializerProvider.currentSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility updateValueSerializer(TypeSerializer newValueSerializer) { + return valueSerializerProvider.registerNewSerializerForRestoredState(newValueSerializer); + } + + @Nullable + public TypeSerializer getPreviousValueSerializer() { + return valueSerializerProvider.previousSchemaSerializer(); } @Nonnull @@ -122,16 +160,16 @@ public boolean equals(Object obj) { return Objects.equals(name, other.getName()) && Objects.equals(assignmentMode, other.getAssignmentMode()) - && Objects.equals(keySerializer, other.getKeySerializer()) - && Objects.equals(valueSerializer, other.getValueSerializer()); + && Objects.equals(getKeySerializer(), other.getKeySerializer()) + && Objects.equals(getValueSerializer(), other.getValueSerializer()); } @Override public int hashCode() { int result = name.hashCode(); result = 31 * result + assignmentMode.hashCode(); - result = 31 * result + keySerializer.hashCode(); - result = 31 * result + valueSerializer.hashCode(); + result = 31 * result + getKeySerializer().hashCode(); + result = 31 * result + getValueSerializer().hashCode(); return result; } @@ -139,8 +177,8 @@ public int hashCode() { public String toString() { return "RegisteredBroadcastBackendStateMetaInfo{" + "name='" + name + '\'' + - ", keySerializer=" + keySerializer + - ", valueSerializer=" + valueSerializer + + ", keySerializer=" + getKeySerializer() + + ", valueSerializer=" + getValueSerializer() + ", assignmentMode=" + assignmentMode + '}'; } @@ -154,8 +192,12 @@ private StateMetaInfoSnapshot computeSnapshot() { Map> serializerConfigSnapshotsMap = new HashMap<>(2); String keySerializerKey = StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER.toString(); String valueSerializerKey = StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString(); + + TypeSerializer keySerializer = getKeySerializer(); serializerMap.put(keySerializerKey, keySerializer.duplicate()); serializerConfigSnapshotsMap.put(keySerializerKey, keySerializer.snapshotConfiguration()); + + TypeSerializer valueSerializer = getValueSerializer(); serializerMap.put(valueSerializerKey, valueSerializer.duplicate()); serializerConfigSnapshotsMap.put(valueSerializerKey, valueSerializer.snapshotConfiguration()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java index d05f31a0c5c76..ebe8e947badd5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; import org.apache.flink.util.Preconditions; @@ -44,18 +45,24 @@ public class RegisteredKeyValueStateBackendMetaInfo extends RegisteredStat @Nonnull private final StateDescriptor.Type stateType; @Nonnull - private final TypeSerializer namespaceSerializer; + private final StateSerializerProvider namespaceSerializerProvider; @Nonnull - private final TypeSerializer stateSerializer; + private final StateSerializerProvider stateSerializerProvider; @Nullable - private final StateSnapshotTransformer snapshotTransformer; + private StateSnapshotTransformer snapshotTransformer; public RegisteredKeyValueStateBackendMetaInfo( @Nonnull StateDescriptor.Type stateType, @Nonnull String name, @Nonnull TypeSerializer namespaceSerializer, @Nonnull TypeSerializer stateSerializer) { - this(stateType, name, namespaceSerializer, stateSerializer, null); + + this( + stateType, + name, + StateSerializerProvider.fromNewState(namespaceSerializer), + StateSerializerProvider.fromNewState(stateSerializer), + null); } public RegisteredKeyValueStateBackendMetaInfo( @@ -65,11 +72,12 @@ public RegisteredKeyValueStateBackendMetaInfo( @Nonnull TypeSerializer stateSerializer, @Nullable StateSnapshotTransformer snapshotTransformer) { - super(name); - this.stateType = stateType; - this.namespaceSerializer = namespaceSerializer; - this.stateSerializer = stateSerializer; - this.snapshotTransformer = snapshotTransformer; + this( + stateType, + name, + StateSerializerProvider.fromNewState(namespaceSerializer), + StateSerializerProvider.fromNewState(stateSerializer), + snapshotTransformer); } @SuppressWarnings("unchecked") @@ -77,13 +85,31 @@ public RegisteredKeyValueStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sna this( StateDescriptor.Type.valueOf(snapshot.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.KEYED_STATE_TYPE)), snapshot.getName(), - (TypeSerializer) Preconditions.checkNotNull( - snapshot.restoreTypeSerializer(StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER)), - (TypeSerializer) Preconditions.checkNotNull( - snapshot.restoreTypeSerializer(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)), null); + StateSerializerProvider.fromRestoredState( + (TypeSerializerSnapshot) Preconditions.checkNotNull( + snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER))), + StateSerializerProvider.fromRestoredState( + (TypeSerializerSnapshot) Preconditions.checkNotNull( + snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))), + null); + Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.KEY_VALUE == snapshot.getBackendStateType()); } + private RegisteredKeyValueStateBackendMetaInfo( + @Nonnull StateDescriptor.Type stateType, + @Nonnull String name, + @Nonnull StateSerializerProvider namespaceSerializerProvider, + @Nonnull StateSerializerProvider stateSerializerProvider, + @Nullable StateSnapshotTransformer snapshotTransformer) { + + super(name); + this.stateType = stateType; + this.namespaceSerializerProvider = namespaceSerializerProvider; + this.stateSerializerProvider = stateSerializerProvider; + this.snapshotTransformer = snapshotTransformer; + } + @Nonnull public StateDescriptor.Type getStateType() { return stateType; @@ -91,12 +117,32 @@ public StateDescriptor.Type getStateType() { @Nonnull public TypeSerializer getNamespaceSerializer() { - return namespaceSerializer; + return namespaceSerializerProvider.currentSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility updateNamespaceSerializer(TypeSerializer newNamespaceSerializer) { + return namespaceSerializerProvider.registerNewSerializerForRestoredState(newNamespaceSerializer); + } + + @Nullable + public TypeSerializer getPreviousNamespaceSerializer() { + return namespaceSerializerProvider.previousSchemaSerializer(); } @Nonnull public TypeSerializer getStateSerializer() { - return stateSerializer; + return stateSerializerProvider.currentSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility updateStateSerializer(TypeSerializer newStateSerializer) { + return stateSerializerProvider.registerNewSerializerForRestoredState(newStateSerializer); + } + + @Nullable + public TypeSerializer getPreviousStateSerializer() { + return stateSerializerProvider.previousSchemaSerializer(); } @Nullable @@ -104,6 +150,10 @@ public StateSnapshotTransformer getSnapshotTransformer() { return snapshotTransformer; } + public void updateSnapshotTransformer(StateSnapshotTransformer snapshotTransformer) { + this.snapshotTransformer = snapshotTransformer; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -133,8 +183,8 @@ public String toString() { return "RegisteredKeyedBackendStateMetaInfo{" + "stateType=" + stateType + ", name='" + name + '\'' + - ", namespaceSerializer=" + namespaceSerializer + - ", stateSerializer=" + stateSerializer + + ", namespaceSerializer=" + getNamespaceSerializer() + + ", stateSerializer=" + getStateSerializer() + '}'; } @@ -153,34 +203,19 @@ public StateMetaInfoSnapshot snapshot() { return computeSnapshot(); } - public static void checkStateMetaInfo(StateMetaInfoSnapshot stateMetaInfoSnapshot, StateDescriptor stateDesc) { - Preconditions.checkState( - stateMetaInfoSnapshot != null, - "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," + - " but its corresponding restored snapshot cannot be found."); - - Preconditions.checkState(stateMetaInfoSnapshot.getBackendStateType() - == StateMetaInfoSnapshot.BackendStateType.KEY_VALUE, - "Incompatible state types. " + - "Was [" + stateMetaInfoSnapshot.getBackendStateType() + "], " + - "registered as [" + StateMetaInfoSnapshot.BackendStateType.KEY_VALUE + "]."); + public void checkStateMetaInfo(StateDescriptor stateDesc) { Preconditions.checkState( - Objects.equals(stateDesc.getName(), stateMetaInfoSnapshot.getName()), + Objects.equals(stateDesc.getName(), getName()), "Incompatible state names. " + - "Was [" + stateMetaInfoSnapshot.getName() + "], " + + "Was [" + getName() + "], " + "registered with [" + stateDesc.getName() + "]."); - final StateDescriptor.Type restoredType = - StateDescriptor.Type.valueOf( - stateMetaInfoSnapshot.getOption( - StateMetaInfoSnapshot.CommonOptionsKeys.KEYED_STATE_TYPE)); - - if (stateDesc.getType() != StateDescriptor.Type.UNKNOWN && restoredType != StateDescriptor.Type.UNKNOWN) { + if (stateDesc.getType() != StateDescriptor.Type.UNKNOWN && getStateType() != StateDescriptor.Type.UNKNOWN) { Preconditions.checkState( - stateDesc.getType() == restoredType, + stateDesc.getType() == getStateType(), "Incompatible key/value state types. " + - "Was [" + restoredType + "], " + + "Was [" + getStateType() + "], " + "registered with [" + stateDesc.getType() + "]."); } } @@ -194,8 +229,12 @@ private StateMetaInfoSnapshot computeSnapshot() { Map> serializerConfigSnapshotsMap = new HashMap<>(2); String namespaceSerializerKey = StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER.toString(); String valueSerializerKey = StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString(); + + TypeSerializer namespaceSerializer = getNamespaceSerializer(); serializerMap.put(namespaceSerializerKey, namespaceSerializer.duplicate()); serializerConfigSnapshotsMap.put(namespaceSerializerKey, namespaceSerializer.snapshotConfiguration()); + + TypeSerializer stateSerializer = getStateSerializer(); serializerMap.put(valueSerializerKey, stateSerializer.duplicate()); serializerConfigSnapshotsMap.put(valueSerializerKey, stateSerializer.snapshotConfiguration()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java index 10ba029605702..afb3d77f778cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java @@ -19,11 +19,13 @@ package org.apache.flink.runtime.state; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; @@ -46,21 +48,22 @@ public class RegisteredOperatorStateBackendMetaInfo extends RegisteredStateMe * The type serializer for the elements in the state list */ @Nonnull - private final TypeSerializer partitionStateSerializer; + private final StateSerializerProvider partitionStateSerializerProvider; public RegisteredOperatorStateBackendMetaInfo( @Nonnull String name, @Nonnull TypeSerializer partitionStateSerializer, @Nonnull OperatorStateHandle.Mode assignmentMode) { - super(name); - this.partitionStateSerializer = partitionStateSerializer; - this.assignmentMode = assignmentMode; + this( + name, + StateSerializerProvider.fromNewState(partitionStateSerializer), + assignmentMode); } private RegisteredOperatorStateBackendMetaInfo(@Nonnull RegisteredOperatorStateBackendMetaInfo copy) { this( Preconditions.checkNotNull(copy).name, - copy.partitionStateSerializer.duplicate(), + copy.getPartitionStateSerializer().duplicate(), copy.assignmentMode); } @@ -68,13 +71,24 @@ private RegisteredOperatorStateBackendMetaInfo(@Nonnull RegisteredOperatorStateB public RegisteredOperatorStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot snapshot) { this( snapshot.getName(), - (TypeSerializer) Preconditions.checkNotNull( - snapshot.restoreTypeSerializer(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)), + StateSerializerProvider.fromRestoredState( + (TypeSerializerSnapshot) Preconditions.checkNotNull( + snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))), OperatorStateHandle.Mode.valueOf( snapshot.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.OPERATOR_STATE_DISTRIBUTION_MODE))); + Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.OPERATOR == snapshot.getBackendStateType()); } + private RegisteredOperatorStateBackendMetaInfo( + @Nonnull String name, + @Nonnull StateSerializerProvider partitionStateSerializerProvider, + @Nonnull OperatorStateHandle.Mode assignmentMode) { + super(name); + this.partitionStateSerializerProvider = partitionStateSerializerProvider; + this.assignmentMode = assignmentMode; + } + /** * Creates a deep copy of the itself. */ @@ -96,7 +110,17 @@ public OperatorStateHandle.Mode getAssignmentMode() { @Nonnull public TypeSerializer getPartitionStateSerializer() { - return partitionStateSerializer; + return partitionStateSerializerProvider.currentSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility updatePartitionStateSerializer(TypeSerializer newPartitionStateSerializer) { + return partitionStateSerializerProvider.registerNewSerializerForRestoredState(newPartitionStateSerializer); + } + + @Nullable + public TypeSerializer getPreviousPartitionStateSerializer() { + return partitionStateSerializerProvider.previousSchemaSerializer(); } @Override @@ -112,7 +136,7 @@ public boolean equals(Object obj) { return (obj instanceof RegisteredOperatorStateBackendMetaInfo) && name.equals(((RegisteredOperatorStateBackendMetaInfo) obj).getName()) && assignmentMode.equals(((RegisteredOperatorStateBackendMetaInfo) obj).getAssignmentMode()) - && partitionStateSerializer.equals(((RegisteredOperatorStateBackendMetaInfo) obj).getPartitionStateSerializer()); + && getPartitionStateSerializer().equals(((RegisteredOperatorStateBackendMetaInfo) obj).getPartitionStateSerializer()); } @Override @@ -128,7 +152,7 @@ public String toString() { return "RegisteredOperatorBackendStateMetaInfo{" + "name='" + name + "\'" + ", assignmentMode=" + assignmentMode + - ", partitionStateSerializer=" + partitionStateSerializer + + ", partitionStateSerializer=" + getPartitionStateSerializer() + '}'; } @@ -138,6 +162,8 @@ private StateMetaInfoSnapshot computeSnapshot() { StateMetaInfoSnapshot.CommonOptionsKeys.OPERATOR_STATE_DISTRIBUTION_MODE.toString(), assignmentMode.toString()); String valueSerializerKey = StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString(); + + TypeSerializer partitionStateSerializer = getPartitionStateSerializer(); Map> serializerMap = Collections.singletonMap(valueSerializerKey, partitionStateSerializer.duplicate()); Map> serializerConfigSnapshotsMap = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java index 0304b929c6d3f..60c88e37f6ad9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java @@ -19,11 +19,13 @@ package org.apache.flink.runtime.state; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot; import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collections; import java.util.Map; @@ -34,24 +36,34 @@ public class RegisteredPriorityQueueStateBackendMetaInfo extends RegisteredStateMetaInfoBase { @Nonnull - private final TypeSerializer elementSerializer; + private final StateSerializerProvider elementSerializerProvider; public RegisteredPriorityQueueStateBackendMetaInfo( @Nonnull String name, @Nonnull TypeSerializer elementSerializer) { - super(name); - this.elementSerializer = elementSerializer; + this(name, StateSerializerProvider.fromNewState(elementSerializer)); } @SuppressWarnings("unchecked") public RegisteredPriorityQueueStateBackendMetaInfo(StateMetaInfoSnapshot snapshot) { - this(snapshot.getName(), - (TypeSerializer) Preconditions.checkNotNull( - snapshot.restoreTypeSerializer(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))); + this( + snapshot.getName(), + StateSerializerProvider.fromRestoredState( + (TypeSerializerSnapshot) Preconditions.checkNotNull( + snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)))); + Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.PRIORITY_QUEUE == snapshot.getBackendStateType()); } + private RegisteredPriorityQueueStateBackendMetaInfo( + @Nonnull String name, + @Nonnull StateSerializerProvider elementSerializerProvider) { + + super(name); + this.elementSerializerProvider = elementSerializerProvider; + } + @Nonnull @Override public StateMetaInfoSnapshot snapshot() { @@ -60,10 +72,21 @@ public StateMetaInfoSnapshot snapshot() { @Nonnull public TypeSerializer getElementSerializer() { - return elementSerializer; + return elementSerializerProvider.currentSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility updateElementSerializer(TypeSerializer newElementSerializer) { + return elementSerializerProvider.registerNewSerializerForRestoredState(newElementSerializer); + } + + @Nullable + public TypeSerializer getPreviousElementSerializer() { + return elementSerializerProvider.previousSchemaSerializer(); } private StateMetaInfoSnapshot computeSnapshot() { + TypeSerializer elementSerializer = getElementSerializer(); Map> serializerMap = Collections.singletonMap( StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString(), @@ -82,6 +105,6 @@ private StateMetaInfoSnapshot computeSnapshot() { } public RegisteredPriorityQueueStateBackendMetaInfo deepCopy() { - return new RegisteredPriorityQueueStateBackendMetaInfo<>(name, elementSerializer.duplicate()); + return new RegisteredPriorityQueueStateBackendMetaInfo<>(name, getElementSerializer().duplicate()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSerializerProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSerializerProvider.java new file mode 100644 index 0000000000000..a24f12e42fbee --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSerializerProvider.java @@ -0,0 +1,245 @@ +/* + * 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.state; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A {@link StateSerializerProvider} wraps logic on how to obtain serializers for registered state, + * either with the previous schema of state in checkpoints or the current schema of state. + * + * @param the type of the state. + */ +@Internal +public abstract class StateSerializerProvider { + + /** + * The registered serializer for the state. + * + *

    In the case that this provider was created from a restored serializer snapshot via + * {@link #fromRestoredState(TypeSerializerSnapshot)}, but a new serializer was never registered + * for the state (i.e., this is the case if a restored state was never accessed), this would be {@code null}. + */ + @Nullable + TypeSerializer registeredSerializer; + + /** + * Creates a {@link StateSerializerProvider} for restored state from the previous serializer's snapshot. + * + *

    Once a new serializer is registered for the state, it should be provided via + * the {@link #registerNewSerializerForRestoredState(TypeSerializer)} method. + * + * @param stateSerializerSnapshot the previous serializer's snapshot. + * @param the type of the state. + * + * @return a new {@link StateSerializerProvider} for restored state. + */ + public static StateSerializerProvider fromRestoredState(TypeSerializerSnapshot stateSerializerSnapshot) { + return new RestoredStateSerializerProvider<>(stateSerializerSnapshot); + } + + /** + * Creates a {@link StateSerializerProvider} for new state from the registered state serializer. + * + * @param registeredStateSerializer the new state's registered serializer. + * @param the type of the state. + * + * @return a new {@link StateSerializerProvider} for new state. + */ + public static StateSerializerProvider fromNewState(TypeSerializer registeredStateSerializer) { + return new NewStateSerializerProvider<>(registeredStateSerializer); + } + + private StateSerializerProvider(@Nullable TypeSerializer stateSerializer) { + this.registeredSerializer = stateSerializer; + } + + /** + * Gets the serializer that recognizes the current serialization schema of the state. + * This is the serializer that should be used for regular state serialization and + * deserialization after state has been restored. + * + *

    If this provider was created from a restored state's serializer snapshot, while a + * new serializer (with a new schema) was not registered for the state (i.e., because + * the state was never accessed after it was restored), then the schema of state remains + * identical. Therefore, in this case, it is guaranteed that the serializer returned by + * this method is the same as the one returned by {@link #previousSchemaSerializer()}. + * + *

    If this provider was created from new state, then this always returns the + * serializer that the new state was registered with. + * + * @return a serializer that reads and writes in the current schema of the state. + */ + @Nonnull + public abstract TypeSerializer currentSchemaSerializer(); + + /** + * Gets the serializer that recognizes the previous serialization schema of the state. + * This is the serializer that should be used for restoring the state, i.e. when the state + * is still in the previous serialization schema. + * + *

    This method can only be used if this provider was created from a restored state's serializer + * snapshot. If this provider was created from new state, then this method is + * irrelevant, since there doesn't exist any previous version of the state schema. + * + * @return a serializer that reads and writes in the previous schema of the state. + */ + @Nonnull + public abstract TypeSerializer previousSchemaSerializer(); + + /** + * For restored state, register a new serializer that potentially has a new serialization schema. + * + *

    Users are allowed to register serializers for state only once. Therefore, this method + * is irrelevant if this provider was created from new state, since a state serializer had + * been registered already. + * + *

    For the case where this provider was created from restored state, then this method should + * be called at most once. The new serializer will be checked for its schema compatibility with the + * previous serializer's schema, and returned to the caller. The caller is responsible for + * checking the result and react appropriately to it, as follows: + *

      + *
    • {@link TypeSerializerSchemaCompatibility#isCompatibleAsIs()}: nothing needs to be done. + * {@link #currentSchemaSerializer()} now returns the newly registered serializer.
    • + *
    • {@link TypeSerializerSchemaCompatibility#isCompatibleAfterMigration()} ()}: state needs to be + * migrated before the serializer returned by {@link #currentSchemaSerializer()} can be used. + * The migration should be performed by reading the state with {@link #previousSchemaSerializer()}, + * and then writing it again with {@link #currentSchemaSerializer()}.
    • + *
    • {@link TypeSerializerSchemaCompatibility#isIncompatible()}: the registered serializer is + * incompatible. {@link #currentSchemaSerializer()} can no longer return a serializer for + * the state, and therefore this provider shouldn't be used anymore.
    • + *
    + * + * @return the schema compatibility of the new registered serializer, with respect to the previous serializer. + */ + @Nonnull + public abstract TypeSerializerSchemaCompatibility registerNewSerializerForRestoredState(TypeSerializer newSerializer); + + /** + * Implementation of the {@link StateSerializerProvider} for the restored state case. + */ + private static class RestoredStateSerializerProvider extends StateSerializerProvider { + + /** + * The snapshot of the previous serializer of the state. + */ + @Nonnull + private final TypeSerializerSnapshot previousSerializerSnapshot; + + private boolean isRegisteredWithIncompatibleSerializer = false; + + RestoredStateSerializerProvider(TypeSerializerSnapshot previousSerializerSnapshot) { + super(null); + this.previousSerializerSnapshot = Preconditions.checkNotNull(previousSerializerSnapshot); + } + + /** + * The restore serializer, lazily created only when the restore serializer is accessed. + * + *

    NOTE: It is important to only create this lazily, so that off-heap + * state do not fail eagerly when restoring state that has a + * {@link UnloadableDummyTypeSerializer} as the previous serializer. This should + * be relevant only for restores from Flink versions prior to 1.7.x. + */ + @Nullable + private TypeSerializer cachedRestoredSerializer; + + @Override + @Nonnull + public TypeSerializer currentSchemaSerializer() { + if (registeredSerializer != null) { + checkState( + !isRegisteredWithIncompatibleSerializer, + "Unable to provide a serializer with the current schema, because the restored state was " + + "registered with a new serializer that has incompatible schema."); + + return registeredSerializer; + } + + // if we are not yet registered with a new serializer, + // we can just use the restore serializer to read / write the state. + return previousSchemaSerializer(); + } + + @Nonnull + public TypeSerializerSchemaCompatibility registerNewSerializerForRestoredState(TypeSerializer newSerializer) { + checkNotNull(newSerializer); + if (registeredSerializer != null) { + throw new UnsupportedOperationException("A serializer has already been registered for the state; re-registration is not allowed."); + } + + TypeSerializerSchemaCompatibility result = previousSerializerSnapshot.resolveSchemaCompatibility(newSerializer); + if (result.isIncompatible()) { + this.isRegisteredWithIncompatibleSerializer = true; + } + this.registeredSerializer = newSerializer; + return result; + } + + @Nonnull + public final TypeSerializer previousSchemaSerializer() { + if (cachedRestoredSerializer != null) { + return cachedRestoredSerializer; + } + + this.cachedRestoredSerializer = previousSerializerSnapshot.restoreSerializer(); + return cachedRestoredSerializer; + } + } + + /** + * Implementation of the {@link StateSerializerProvider} for the new state case. + */ + private static class NewStateSerializerProvider extends StateSerializerProvider { + + NewStateSerializerProvider(TypeSerializer registeredStateSerializer) { + super(Preconditions.checkNotNull(registeredStateSerializer)); + } + + @Override + @Nonnull + @SuppressWarnings("ConstantConditions") + public TypeSerializer currentSchemaSerializer() { + return registeredSerializer; + } + + @Override + @Nonnull + public TypeSerializerSchemaCompatibility registerNewSerializerForRestoredState(TypeSerializer newSerializer) { + throw new UnsupportedOperationException("A serializer has already been registered for the state; re-registration is not allowed."); + } + + @Override + @Nonnull + public TypeSerializer previousSchemaSerializer() { + throw new UnsupportedOperationException("This is a NewStateSerializerProvider; you cannot get a restore serializer because there was no restored state."); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index 54cac1ea17746..8708613e2a3cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -3145,6 +3145,57 @@ public void testMapStateDefaultValue() throws Exception { backend.dispose(); } + @Test + public void testSnapshotNonAccessedState() throws Exception { + CheckpointStreamFactory streamFactory = createStreamFactory(); + SharedStateRegistry sharedStateRegistry = new SharedStateRegistry(); + AbstractKeyedStateBackend backend = createKeyedBackend(StringSerializer.INSTANCE); + + final String stateName = "test-name"; + try { + MapStateDescriptor kvId = new MapStateDescriptor<>(stateName, Integer.class, String.class); + MapState mapState = backend + .getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + + // write some state to be snapshotted + backend.setCurrentKey("1"); + mapState.put(11, "foo"); + backend.setCurrentKey("2"); + mapState.put(8, "bar"); + backend.setCurrentKey("3"); + mapState.put(91, "hello world"); + + // take a snapshot, and then restore backend with snapshot + KeyedStateHandle snapshot = runSnapshot( + backend.snapshot(1L, 2L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()), + sharedStateRegistry); + backend.dispose(); + + backend = restoreKeyedBackend(StringSerializer.INSTANCE, snapshot); + + // now take a snapshot again without accessing the state + snapshot = runSnapshot( + backend.snapshot(2L, 3L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()), + sharedStateRegistry); + backend.dispose(); + + // we restore again and try to access previous state + backend = restoreKeyedBackend(StringSerializer.INSTANCE, snapshot); + mapState = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + + backend.setCurrentKey("1"); + assertEquals("foo", mapState.get(11)); + backend.setCurrentKey("2"); + assertEquals("bar", mapState.get(8)); + backend.setCurrentKey("3"); + assertEquals("hello world", mapState.get(91)); + + snapshot.discardState(); + } finally { + backend.dispose(); + } + } + /** * This test verifies that state is correctly assigned to key groups and that restore * restores the relevant key groups in the backend. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSerializerProviderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSerializerProviderTest.java new file mode 100644 index 0000000000000..de1f2bd962cd2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSerializerProviderTest.java @@ -0,0 +1,187 @@ +/* + * 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.state; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.testutils.statemigration.TestType; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Test suit for {@link StateSerializerProvider}. + */ +public class StateSerializerProviderTest { + + // -------------------------------------------------------------------------------- + // Tests for #currentSchemaSerializer() + // -------------------------------------------------------------------------------- + + @Test + public void testCurrentSchemaSerializerForNewStateSerializerProvider() { + StateSerializerProvider testProvider = StateSerializerProvider.fromNewState(new TestType.V1TestTypeSerializer()); + assertTrue(testProvider.currentSchemaSerializer() instanceof TestType.V1TestTypeSerializer); + } + + @Test + public void testCurrentSchemaSerializerForRestoredStateSerializerProvider() { + TestType.V1TestTypeSerializer serializer = new TestType.V1TestTypeSerializer(); + StateSerializerProvider testProvider = StateSerializerProvider.fromRestoredState(serializer.snapshotConfiguration()); + assertTrue(testProvider.currentSchemaSerializer() instanceof TestType.V1TestTypeSerializer); + } + + // -------------------------------------------------------------------------------- + // Tests for #previousSchemaSerializer() + // -------------------------------------------------------------------------------- + + @Test(expected = UnsupportedOperationException.class) + public void testPreviousSchemaSerializerForNewStateSerializerProvider() { + StateSerializerProvider testProvider = StateSerializerProvider.fromNewState(new TestType.V1TestTypeSerializer()); + + // this should fail with an exception + testProvider.previousSchemaSerializer(); + } + + @Test + public void testPreviousSchemaSerializerForRestoredStateSerializerProvider() { + TestType.V1TestTypeSerializer serializer = new TestType.V1TestTypeSerializer(); + StateSerializerProvider testProvider = StateSerializerProvider.fromRestoredState(serializer.snapshotConfiguration()); + assertTrue(testProvider.previousSchemaSerializer() instanceof TestType.V1TestTypeSerializer); + } + + @Test + public void testLazyInstantiationOfPreviousSchemaSerializer() { + // create the provider with an exception throwing snapshot; + // this would throw an exception if the restore serializer was eagerly accessed + StateSerializerProvider testProvider = + StateSerializerProvider.fromRestoredState(new ExceptionThrowingSerializerSnapshot()); + + try { + // if we fail here, that means the restore serializer was indeed lazily accessed + testProvider.previousSchemaSerializer(); + fail("expected to fail when accessing the restore serializer."); + } catch (Exception expected) { + // success + } + } + + // -------------------------------------------------------------------------------- + // Tests for #registerNewSerializerForRestoredState(TypeSerializer) + // -------------------------------------------------------------------------------- + + @Test(expected = UnsupportedOperationException.class) + public void testRegisterNewSerializerWithNewStateSerializerProviderShouldFail() { + StateSerializerProvider testProvider = StateSerializerProvider.fromNewState(new TestType.V1TestTypeSerializer()); + testProvider.registerNewSerializerForRestoredState(new TestType.V2TestTypeSerializer()); + } + + @Test(expected = UnsupportedOperationException.class) + public void testRegisterNewSerializerTwiceWithNewStateSerializerProviderShouldFail() { + TestType.V1TestTypeSerializer serializer = new TestType.V1TestTypeSerializer(); + StateSerializerProvider testProvider = StateSerializerProvider.fromRestoredState(serializer.snapshotConfiguration()); + + testProvider.registerNewSerializerForRestoredState(new TestType.V2TestTypeSerializer()); + + // second registration should fail + testProvider.registerNewSerializerForRestoredState(new TestType.V2TestTypeSerializer()); + } + + @Test + public void testRegisterNewCompatibleAsIsSerializer() { + TestType.V1TestTypeSerializer serializer = new TestType.V1TestTypeSerializer(); + StateSerializerProvider testProvider = StateSerializerProvider.fromRestoredState(serializer.snapshotConfiguration()); + + // register compatible serializer for state + TypeSerializerSchemaCompatibility schemaCompatibility = + testProvider.registerNewSerializerForRestoredState(new TestType.V1TestTypeSerializer()); + assertTrue(schemaCompatibility.isCompatibleAsIs()); + + assertTrue(testProvider.currentSchemaSerializer() instanceof TestType.V1TestTypeSerializer); + assertTrue(testProvider.previousSchemaSerializer() instanceof TestType.V1TestTypeSerializer); + } + + @Test + public void testRegisterNewCompatibleAfterMigrationSerializer() { + TestType.V1TestTypeSerializer serializer = new TestType.V1TestTypeSerializer(); + StateSerializerProvider testProvider = StateSerializerProvider.fromRestoredState(serializer.snapshotConfiguration()); + + // register serializer that requires migration for state + TypeSerializerSchemaCompatibility schemaCompatibility = + testProvider.registerNewSerializerForRestoredState(new TestType.V2TestTypeSerializer()); + assertTrue(schemaCompatibility.isCompatibleAfterMigration()); + } + + @Test + public void testRegisterIncompatibleSerializer() { + TestType.V1TestTypeSerializer serializer = new TestType.V1TestTypeSerializer(); + StateSerializerProvider testProvider = StateSerializerProvider.fromRestoredState(serializer.snapshotConfiguration()); + + // register serializer that requires migration for state + TypeSerializerSchemaCompatibility schemaCompatibility = + testProvider.registerNewSerializerForRestoredState(new TestType.IncompatibleTestTypeSerializer()); + assertTrue(schemaCompatibility.isIncompatible()); + + try { + // a serializer for the current schema will no longer be accessible + testProvider.currentSchemaSerializer(); + } catch (Exception excepted) { + // success + } + } + + // -------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------- + + public static class ExceptionThrowingSerializerSnapshot implements TypeSerializerSnapshot { + + @Override + public TypeSerializer restoreSerializer() { + throw new UnsupportedOperationException(); + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { + throw new UnsupportedOperationException(); + } + + @Override + public int getCurrentVersion() { + throw new UnsupportedOperationException(); + } + } +} diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index d12370bd7f65d..065213b176d5a 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -29,7 +29,6 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; @@ -1377,78 +1376,69 @@ private Tuple2 namespaceSerializer, @Nullable StateSnapshotTransformer snapshotTransformer) throws Exception { - Tuple2 stateInfo = + Tuple2 oldStateInfo = kvStateInformation.get(stateDesc.getName()); TypeSerializer stateSerializer = stateDesc.getSerializer(); - RegisteredKeyValueStateBackendMetaInfo newMetaInfo = new RegisteredKeyValueStateBackendMetaInfo<>( - stateDesc.getType(), - stateDesc.getName(), - namespaceSerializer, - stateSerializer, - snapshotTransformer); - - if (stateInfo != null) { - newMetaInfo = migrateStateIfNecessary( - newMetaInfo, + + ColumnFamilyHandle newColumnFamily; + RegisteredKeyValueStateBackendMetaInfo newMetaInfo; + if (oldStateInfo != null) { + @SuppressWarnings("unchecked") + RegisteredKeyValueStateBackendMetaInfo castedMetaInfo = (RegisteredKeyValueStateBackendMetaInfo) oldStateInfo.f1; + + newMetaInfo = updateRestoredStateMetaInfo( + Tuple2.of(oldStateInfo.f0, castedMetaInfo), stateDesc, namespaceSerializer, stateSerializer, - stateInfo); + snapshotTransformer); - stateInfo.f1 = newMetaInfo; + oldStateInfo.f1 = newMetaInfo; + newColumnFamily = oldStateInfo.f0; } else { - ColumnFamilyHandle columnFamily = createColumnFamily(stateDesc.getName()); + newMetaInfo = new RegisteredKeyValueStateBackendMetaInfo<>( + stateDesc.getType(), + stateDesc.getName(), + namespaceSerializer, + stateSerializer, + snapshotTransformer); - stateInfo = Tuple2.of(columnFamily, newMetaInfo); - registerKvStateInformation(stateDesc.getName(), stateInfo); + newColumnFamily = createColumnFamily(stateDesc.getName()); + registerKvStateInformation(stateDesc.getName(), Tuple2.of(newColumnFamily, newMetaInfo)); } - return Tuple2.of(stateInfo.f0, newMetaInfo); + return Tuple2.of(newColumnFamily, newMetaInfo); } - private RegisteredKeyValueStateBackendMetaInfo migrateStateIfNecessary( - RegisteredKeyValueStateBackendMetaInfo newMetaInfo, + private RegisteredKeyValueStateBackendMetaInfo updateRestoredStateMetaInfo( + Tuple2> oldStateInfo, StateDescriptor stateDesc, TypeSerializer namespaceSerializer, TypeSerializer stateSerializer, - Tuple2 stateInfo) throws Exception { - - StateMetaInfoSnapshot restoredMetaInfoSnapshot = restoredKvStateMetaInfos.get(stateDesc.getName()); - - Preconditions.checkState( - restoredMetaInfoSnapshot != null, - "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," + - " but its corresponding restored snapshot cannot be found."); + @Nullable StateSnapshotTransformer snapshotTransformer) throws Exception { @SuppressWarnings("unchecked") - TypeSerializerSnapshot namespaceSerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot( - StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER.toString())); + RegisteredKeyValueStateBackendMetaInfo restoredKvStateMetaInfo = oldStateInfo.f1; + + restoredKvStateMetaInfo.updateSnapshotTransformer(snapshotTransformer); - TypeSerializerSchemaCompatibility namespaceCompatibility = - namespaceSerializerSnapshot.resolveSchemaCompatibility(namespaceSerializer); - if (!namespaceCompatibility.isCompatibleAsIs()) { + TypeSerializerSchemaCompatibility s = restoredKvStateMetaInfo.updateNamespaceSerializer(namespaceSerializer); + if (!s.isCompatibleAsIs()) { throw new StateMigrationException("The new namespace serializer must be compatible."); } - @SuppressWarnings("unchecked") - TypeSerializerSnapshot stateSerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot( - StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString())); - - RegisteredKeyValueStateBackendMetaInfo.checkStateMetaInfo(restoredMetaInfoSnapshot, stateDesc); + restoredKvStateMetaInfo.checkStateMetaInfo(stateDesc); - TypeSerializerSchemaCompatibility stateCompatibility = - stateSerializerSnapshot.resolveSchemaCompatibility(stateSerializer); - - if (stateCompatibility.isCompatibleAfterMigration()) { - migrateStateValues(stateDesc, stateInfo, restoredMetaInfoSnapshot, newMetaInfo, stateSerializer); - } else if (stateCompatibility.isIncompatible()) { + TypeSerializerSchemaCompatibility newStateSerializerCompatibility = + restoredKvStateMetaInfo.updateStateSerializer(stateSerializer); + if (newStateSerializerCompatibility.isCompatibleAfterMigration()) { + migrateStateValues(stateDesc, oldStateInfo); + } else if (newStateSerializerCompatibility.isIncompatible()) { throw new StateMigrationException("The new state serializer cannot be incompatible."); } - return newMetaInfo; + return restoredKvStateMetaInfo; } /** @@ -1458,10 +1448,7 @@ private RegisteredKeyValueStateBackendMetaInfo m */ private void migrateStateValues( StateDescriptor stateDesc, - Tuple2 stateInfo, - StateMetaInfoSnapshot restoredMetaInfoSnapshot, - RegisteredKeyValueStateBackendMetaInfo newMetaInfo, - TypeSerializer newStateSerializer) throws Exception { + Tuple2> stateMetaInfo) throws Exception { if (stateDesc.getType() == StateDescriptor.Type.MAP) { throw new StateMigrationException("The new serializer for a MapState requires state migration in order for the job to proceed." + @@ -1483,7 +1470,7 @@ private void migrateStateValues( } State state = stateFactory.createState( stateDesc, - Tuple2.of(stateInfo.f0, newMetaInfo), + stateMetaInfo, RocksDBKeyedStateBackend.this); if (!(state instanceof AbstractRocksDBState)) { throw new FlinkRuntimeException( @@ -1495,16 +1482,11 @@ private void migrateStateValues( Snapshot rocksDBSnapshot = db.getSnapshot(); try ( - RocksIteratorWrapper iterator = getRocksIterator(db, stateInfo.f0); + RocksIteratorWrapper iterator = getRocksIterator(db, stateMetaInfo.f0); RocksDBWriteBatchWrapper batchWriter = new RocksDBWriteBatchWrapper(db, getWriteOptions()) ) { iterator.seekToFirst(); - @SuppressWarnings("unchecked") - TypeSerializerSnapshot priorValueSerializerSnapshot = (TypeSerializerSnapshot) - Preconditions.checkNotNull(restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)); - TypeSerializer priorValueSerializer = priorValueSerializerSnapshot.restoreSerializer(); - DataInputDeserializer serializedValueInput = new DataInputDeserializer(); DataOutputSerializer migratedSerializedValueOutput = new DataOutputSerializer(512); while (iterator.isValid()) { @@ -1513,10 +1495,10 @@ private void migrateStateValues( rocksDBState.migrateSerializedValue( serializedValueInput, migratedSerializedValueOutput, - priorValueSerializer, - newStateSerializer); + stateMetaInfo.f1.getPreviousStateSerializer(), + stateMetaInfo.f1.getStateSerializer()); - batchWriter.put(stateInfo.f0, iterator.key(), migratedSerializedValueOutput.getCopyOfBuffer()); + batchWriter.put(stateMetaInfo.f0, iterator.key(), migratedSerializedValueOutput.getCopyOfBuffer()); migratedSerializedValueOutput.clear(); iterator.next(); @@ -1710,25 +1692,16 @@ private Tuple2 tryRegisterP // TODO we implement the simple way of supporting the current functionality, mimicking keyed state // because this should be reworked in FLINK-9376 and then we should have a common algorithm over // StateMetaInfoSnapshot that avoids this code duplication. - StateMetaInfoSnapshot restoredMetaInfoSnapshot = restoredKvStateMetaInfos.get(stateName); - - Preconditions.checkState( - restoredMetaInfoSnapshot != null, - "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," + - " but its corresponding restored snapshot cannot be found."); - StateMetaInfoSnapshot.CommonSerializerKeys serializerKey = - StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER; - - TypeSerializer metaInfoTypeSerializer = restoredMetaInfoSnapshot.restoreTypeSerializer(serializerKey); + @SuppressWarnings("unchecked") + RegisteredPriorityQueueStateBackendMetaInfo castedMetaInfo = + (RegisteredPriorityQueueStateBackendMetaInfo) metaInfoTuple.f1; - if (metaInfoTypeSerializer != byteOrderedElementSerializer) { - @SuppressWarnings("unchecked") - TypeSerializerSnapshot serializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot(serializerKey)); + TypeSerializer previousElementSerializer = castedMetaInfo.getPreviousElementSerializer(); + if (previousElementSerializer != byteOrderedElementSerializer) { TypeSerializerSchemaCompatibility compatibilityResult = - serializerSnapshot.resolveSchemaCompatibility(byteOrderedElementSerializer); + castedMetaInfo.updateElementSerializer(byteOrderedElementSerializer); // Since priority queue elements are written into RocksDB // as keys prefixed with the key group and namespace, we do not support From c8860074ffca82bf0314c8065b62901a8a4cabdd Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 10 Dec 2018 15:07:47 +0800 Subject: [PATCH 212/359] [FLINK-11094] [state backends] State backends no longer need separate map for restored StateMetaInfoSnapshots Since now all restored state meta info snapshots are handled so that we always eagerly create the corresponding RegisteredStateMetaInfoBase for it, the information is already part of the registered state infos map. As can be seen in the changes, those maps are no longer queried and can therefore be safely removed. This closes #7264. --- .../state/DefaultOperatorStateBackend.java | 58 +++------------ .../state/heap/HeapKeyedStateBackend.java | 74 ++++--------------- .../state/RocksDBKeyedStateBackend.java | 15 ---- 3 files changed, 26 insertions(+), 121 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java index 4702919384fad..952dffbc70f19 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java @@ -26,7 +26,6 @@ import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.FSDataInputStream; @@ -105,23 +104,10 @@ public class DefaultOperatorStateBackend implements OperatorStateBackend { */ private final boolean asynchronousSnapshots; - /** - * Map of state names to their corresponding restored state meta info. - * - *

    TODO this map can be removed when eager-state registration is in place. - * TODO we currently need this cached to check state migration strategies when new serializers are registered. - */ - private final Map restoredOperatorStateMetaInfos; - - /** - * Map of state names to their corresponding restored broadcast state meta info. - */ - private final Map restoredBroadcastStateMetaInfos; - /** * Cache of already accessed states. * - *

    In contrast to {@link #registeredOperatorStates} and {@link #restoredOperatorStateMetaInfos} which may be repopulated + *

    In contrast to {@link #registeredOperatorStates} which may be repopulated * with restored state, this map is always empty at the beginning. * *

    TODO this map should be moved to a base class once we have proper hierarchy for the operator state backends. @@ -148,8 +134,6 @@ public DefaultOperatorStateBackend( this.asynchronousSnapshots = asynchronousSnapshots; this.accessedStatesByName = new HashMap<>(); this.accessedBroadcastStatesByName = new HashMap<>(); - this.restoredOperatorStateMetaInfos = new HashMap<>(); - this.restoredBroadcastStateMetaInfos = new HashMap<>(); this.snapshotStrategy = new DefaultOperatorStateBackendSnapshotStrategy(); } @@ -226,34 +210,22 @@ public BroadcastState getBroadcastState(final MapStateDescriptor restoredBroadcastStateMetaInfo = broadcastState.getStateMetaInfo(); // check whether new serializers are incompatible - TypeSerializerSnapshot keySerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) metaInfoSnapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER)); - TypeSerializerSchemaCompatibility keyCompatibility = - keySerializerSnapshot.resolveSchemaCompatibility(broadcastStateKeySerializer); + restoredBroadcastStateMetaInfo.updateKeySerializer(broadcastStateKeySerializer); if (keyCompatibility.isIncompatible()) { throw new StateMigrationException("The new key serializer for broadcast state must not be incompatible."); } - TypeSerializerSnapshot valueSerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) metaInfoSnapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)); - TypeSerializerSchemaCompatibility valueCompatibility = - valueSerializerSnapshot.resolveSchemaCompatibility(broadcastStateValueSerializer); + restoredBroadcastStateMetaInfo.updateValueSerializer(broadcastStateValueSerializer); if (valueCompatibility.isIncompatible()) { throw new StateMigrationException("The new value serializer for broadcast state must not be incompatible."); } - // new serializer is compatible; use it to replace the old serializer - broadcastState.setStateMetaInfo( - new RegisteredBroadcastStateBackendMetaInfo<>( - name, - OperatorStateHandle.Mode.BROADCAST, - broadcastStateKeySerializer, - broadcastStateValueSerializer)); + broadcastState.setStateMetaInfo(restoredBroadcastStateMetaInfo); } accessedBroadcastStatesByName.put(name, broadcastState); @@ -345,8 +317,6 @@ public void restore(Collection restoreSnapshots) throws Exc " not be loaded. This is a temporary restriction that will be fixed in future versions."); } - restoredOperatorStateMetaInfos.put(restoredSnapshot.getName(), restoredSnapshot); - PartitionableListState listState = registeredOperatorStates.get(restoredSnapshot.getName()); if (null == listState) { @@ -381,8 +351,6 @@ public void restore(Collection restoreSnapshots) throws Exc " not be loaded. This is a temporary restriction that will be fixed in future versions."); } - restoredBroadcastStateMetaInfos.put(restoredSnapshot.getName(), restoredSnapshot); - BackendWritableBroadcastState broadcastState = registeredBroadcastStates.get(restoredSnapshot.getName()); if (broadcastState == null) { @@ -590,25 +558,19 @@ private ListState getListState( partitionableListState.getStateMetaInfo().getAssignmentMode(), mode); - StateMetaInfoSnapshot restoredSnapshot = restoredOperatorStateMetaInfos.get(name); - RegisteredOperatorStateBackendMetaInfo metaInfo = - new RegisteredOperatorStateBackendMetaInfo<>(restoredSnapshot); + RegisteredOperatorStateBackendMetaInfo restoredPartitionableListStateMetaInfo = + partitionableListState.getStateMetaInfo(); - // check compatibility to determine if state migration is required + // check compatibility to determine if new serializers are incompatible TypeSerializer newPartitionStateSerializer = partitionStateSerializer.duplicate(); - @SuppressWarnings("unchecked") - TypeSerializerSnapshot stateSerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredSnapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)); - TypeSerializerSchemaCompatibility stateCompatibility = - stateSerializerSnapshot.resolveSchemaCompatibility(newPartitionStateSerializer); + restoredPartitionableListStateMetaInfo.updatePartitionStateSerializer(newPartitionStateSerializer); if (stateCompatibility.isIncompatible()) { throw new StateMigrationException("The new state serializer for operator state must not be incompatible."); } - partitionableListState.setStateMetaInfo( - new RegisteredOperatorStateBackendMetaInfo<>(name, newPartitionStateSerializer, mode)); + partitionableListState.setStateMetaInfo(restoredPartitionableListStateMetaInfo); } accessedStatesByName.put(name, partitionableListState); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java index ecad76c798f37..3f8761b657a3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java @@ -30,7 +30,6 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.memory.DataInputViewStreamWrapper; @@ -79,6 +78,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.io.InputStream; @@ -125,14 +125,6 @@ public class HeapKeyedStateBackend extends AbstractKeyedStateBackend { */ private final Map registeredPQStates; - /** - * Map of state names to their corresponding restored state meta info. - * - *

    TODO this map can be removed when eager-state registration is in place. - * TODO we currently need this cached to check state migration strategies when new serializers are registered. - */ - private final Map restoredStateMetaInfo; - /** * The configuration for local recovery. */ @@ -173,7 +165,6 @@ public HeapKeyedStateBackend( this.snapshotStrategy = new HeapSnapshotStrategy(synchronicityTrait); LOG.info("Initializing heap keyed state backend with stream factory."); - this.restoredStateMetaInfo = new HashMap<>(); this.priorityQueueSetFactory = priorityQueueSetFactory; } @@ -194,23 +185,9 @@ public KeyGrou // TODO we implement the simple way of supporting the current functionality, mimicking keyed state // because this should be reworked in FLINK-9376 and then we should have a common algorithm over // StateMetaInfoSnapshot that avoids this code duplication. - StateMetaInfoSnapshot restoredMetaInfoSnapshot = - restoredStateMetaInfo.get(StateUID.of(stateName, StateMetaInfoSnapshot.BackendStateType.PRIORITY_QUEUE)); - - Preconditions.checkState( - restoredMetaInfoSnapshot != null, - "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," + - " but its corresponding restored snapshot cannot be found."); - - StateMetaInfoSnapshot.CommonSerializerKeys serializerKey = - StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER; - - @SuppressWarnings("unchecked") - TypeSerializerSnapshot serializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot(serializerKey)); TypeSerializerSchemaCompatibility compatibilityResult = - serializerSnapshot.resolveSchemaCompatibility(byteOrderedElementSerializer); + existingState.getMetaInfo().updateElementSerializer(byteOrderedElementSerializer); if (compatibilityResult.isIncompatible()) { throw new FlinkRuntimeException(new StateMigrationException("For heap backends, the new priority queue serializer must not be incompatible.")); @@ -252,57 +229,42 @@ private KeyGro private StateTable tryRegisterStateTable( TypeSerializer namespaceSerializer, StateDescriptor stateDesc, - StateSnapshotTransformer snapshotTransformer) throws StateMigrationException { + @Nullable StateSnapshotTransformer snapshotTransformer) throws StateMigrationException { @SuppressWarnings("unchecked") StateTable stateTable = (StateTable) registeredKVStates.get(stateDesc.getName()); TypeSerializer newStateSerializer = stateDesc.getSerializer(); - RegisteredKeyValueStateBackendMetaInfo newMetaInfo = new RegisteredKeyValueStateBackendMetaInfo<>( - stateDesc.getType(), - stateDesc.getName(), - namespaceSerializer, - newStateSerializer, - snapshotTransformer); if (stateTable != null) { - @SuppressWarnings("unchecked") - StateMetaInfoSnapshot restoredMetaInfoSnapshot = - restoredStateMetaInfo.get( - StateUID.of(stateDesc.getName(), StateMetaInfoSnapshot.BackendStateType.KEY_VALUE)); - - Preconditions.checkState( - restoredMetaInfoSnapshot != null, - "Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," + - " but its corresponding restored snapshot cannot be found."); + RegisteredKeyValueStateBackendMetaInfo restoredKvMetaInfo = stateTable.getMetaInfo(); - @SuppressWarnings("unchecked") - TypeSerializerSnapshot namespaceSerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot( - StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER.toString())); + restoredKvMetaInfo.updateSnapshotTransformer(snapshotTransformer); TypeSerializerSchemaCompatibility namespaceCompatibility = - namespaceSerializerSnapshot.resolveSchemaCompatibility(namespaceSerializer); + restoredKvMetaInfo.updateNamespaceSerializer(namespaceSerializer); if (!namespaceCompatibility.isCompatibleAsIs()) { throw new StateMigrationException("For heap backends, the new namespace serializer must be compatible."); } - @SuppressWarnings("unchecked") - TypeSerializerSnapshot stateSerializerSnapshot = Preconditions.checkNotNull( - (TypeSerializerSnapshot) restoredMetaInfoSnapshot.getTypeSerializerConfigSnapshot( - StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString())); - - RegisteredKeyValueStateBackendMetaInfo.checkStateMetaInfo(restoredMetaInfoSnapshot, stateDesc); + restoredKvMetaInfo.checkStateMetaInfo(stateDesc); TypeSerializerSchemaCompatibility stateCompatibility = - stateSerializerSnapshot.resolveSchemaCompatibility(newStateSerializer); + restoredKvMetaInfo.updateStateSerializer(newStateSerializer); if (stateCompatibility.isIncompatible()) { throw new StateMigrationException("For heap backends, the new state serializer must not be incompatible."); } - stateTable.setMetaInfo(newMetaInfo); + stateTable.setMetaInfo(restoredKvMetaInfo); } else { + RegisteredKeyValueStateBackendMetaInfo newMetaInfo = new RegisteredKeyValueStateBackendMetaInfo<>( + stateDesc.getType(), + stateDesc.getName(), + namespaceSerializer, + newStateSerializer, + snapshotTransformer); + stateTable = snapshotStrategy.newStateTable(newMetaInfo); registeredKVStates.put(stateDesc.getName(), stateTable); } @@ -536,10 +498,6 @@ private void createOrCheckStateForMetaInfo( Map kvStatesById) { for (StateMetaInfoSnapshot metaInfoSnapshot : restoredMetaInfo) { - restoredStateMetaInfo.put( - StateUID.of(metaInfoSnapshot.getName(), metaInfoSnapshot.getBackendStateType()), - metaInfoSnapshot); - final StateSnapshotRestore registeredState; switch (metaInfoSnapshot.getBackendStateType()) { diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 065213b176d5a..a37f8aa8df827 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -111,7 +111,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -212,14 +211,6 @@ IS createState( */ private final LinkedHashMap> kvStateInformation; - /** - * Map of state names to their corresponding restored state meta info. - * - *

    TODO this map can be removed when eager-state registration is in place. - * TODO we currently need this cached to check state migration strategies when new serializers are registered. - */ - private final Map restoredKvStateMetaInfos; - /** Number of bytes required to prefix the key groups. */ private final int keyGroupPrefixBytes; @@ -296,7 +287,6 @@ public RocksDBKeyedStateBackend( this.keyGroupPrefixBytes = RocksDBKeySerializationUtils.computeRequiredBytesInKeyGroupPrefix(getNumberOfKeyGroups()); this.kvStateInformation = new LinkedHashMap<>(); - this.restoredKvStateMetaInfos = new HashMap<>(); this.writeOptions = new WriteOptions().setDisableWAL(true); @@ -424,7 +414,6 @@ public void dispose() { IOUtils.closeQuietly(dbOptions); IOUtils.closeQuietly(writeOptions); kvStateInformation.clear(); - restoredKvStateMetaInfos.clear(); cleanInstanceBasePath(); } @@ -510,7 +499,6 @@ public void restore(Collection restoreState) throws Exception // clear all meta data kvStateInformation.clear(); - restoredKvStateMetaInfos.clear(); try { RocksDBIncrementalRestoreOperation incrementalRestoreOperation = null; @@ -753,8 +741,6 @@ private void restoreKVStateMetaData() throws IOException, StateMigrationExceptio nameBytes, rocksDBKeyedStateBackend.columnOptions); - rocksDBKeyedStateBackend.restoredKvStateMetaInfos.put(restoredMetaInfo.getName(), restoredMetaInfo); - ColumnFamilyHandle columnFamily = rocksDBKeyedStateBackend.db.createColumnFamily(columnFamilyDescriptor); // create a meta info for the state on restore; @@ -1166,7 +1152,6 @@ private List createAndRegisterColumnFamilyDescriptors( stateBackend.columnOptions); columnFamilyDescriptors.add(columnFamilyDescriptor); - stateBackend.restoredKvStateMetaInfos.put(stateMetaInfoSnapshot.getName(), stateMetaInfoSnapshot); } return columnFamilyDescriptors; } From 3f0820a812c8746e61ce13168cc6d0a153c4d357 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 9 Dec 2018 16:08:07 +0800 Subject: [PATCH 213/359] [hotfix] [tests] Remove unused enum from StateBackendTestBase --- .../apache/flink/runtime/state/StateBackendTestBase.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index 8708613e2a3cc..a9bd1f694a30a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -149,14 +149,6 @@ public abstract class StateBackendTestBase exten @Rule public final ExpectedException expectedException = ExpectedException.none(); - /** - * The serialization timeliness behaviour of the state backend under test. - */ - public enum BackendSerializationTimeliness { - ON_ACCESS, - ON_CHECKPOINTS - } - // lazily initialized stream storage private CheckpointStorageLocation checkpointStorageLocation; From 5cc046124863848cecbe982922ab4f62d3d4ee36 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 9 Dec 2018 16:10:31 +0800 Subject: [PATCH 214/359] [hotfix] Cleanup unused methods / appropriate method renames in StateMetaInfoSnapshot This commit removes the `restoreTypeSerializer(...)` method. That method is no longer used after the series of changes in FLINK-11094. This also corresponds to the new principle that the restore serializer is only accessed, when the state backends attempt request it from their state meta infos. We do not create restore serializers eagerly when creating meta infos from a StateMetaInfoSnapshot. It also removes "config" from names of methods and fields related to serializer snapshotting. This corresponds to the abstraction rework of retiring TypeSerializerConfigSnapshot to be replaced by TypeSerializerSnapshot. The related fields / methods should not mention "config" anymore. --- ...gisteredBroadcastStateBackendMetaInfo.java | 4 +-- ...egisteredKeyValueStateBackendMetaInfo.java | 4 +-- ...egisteredOperatorStateBackendMetaInfo.java | 2 +- ...eredPriorityQueueStateBackendMetaInfo.java | 2 +- .../state/metainfo/StateMetaInfoSnapshot.java | 33 +++++++------------ .../StateMetaInfoSnapshotReadersWriters.java | 2 +- .../state/SerializationProxiesTest.java | 4 +-- 7 files changed, 20 insertions(+), 31 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java index 95a650ef3d33c..ecc13faa43d22 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastStateBackendMetaInfo.java @@ -75,10 +75,10 @@ public RegisteredBroadcastStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sn snapshot.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.OPERATOR_STATE_DISTRIBUTION_MODE)), StateSerializerProvider.fromRestoredState( (TypeSerializerSnapshot) Preconditions.checkNotNull( - snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER))), + snapshot.getTypeSerializerSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER))), StateSerializerProvider.fromRestoredState( (TypeSerializerSnapshot) Preconditions.checkNotNull( - snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)))); + snapshot.getTypeSerializerSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)))); Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.BROADCAST == snapshot.getBackendStateType()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java index ebe8e947badd5..b37c79de0264e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyValueStateBackendMetaInfo.java @@ -87,10 +87,10 @@ public RegisteredKeyValueStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sna snapshot.getName(), StateSerializerProvider.fromRestoredState( (TypeSerializerSnapshot) Preconditions.checkNotNull( - snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER))), + snapshot.getTypeSerializerSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.NAMESPACE_SERIALIZER))), StateSerializerProvider.fromRestoredState( (TypeSerializerSnapshot) Preconditions.checkNotNull( - snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))), + snapshot.getTypeSerializerSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))), null); Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.KEY_VALUE == snapshot.getBackendStateType()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java index afb3d77f778cd..921947a4dd0bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorStateBackendMetaInfo.java @@ -73,7 +73,7 @@ public RegisteredOperatorStateBackendMetaInfo(@Nonnull StateMetaInfoSnapshot sna snapshot.getName(), StateSerializerProvider.fromRestoredState( (TypeSerializerSnapshot) Preconditions.checkNotNull( - snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))), + snapshot.getTypeSerializerSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER))), OperatorStateHandle.Mode.valueOf( snapshot.getOption(StateMetaInfoSnapshot.CommonOptionsKeys.OPERATOR_STATE_DISTRIBUTION_MODE))); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java index 60c88e37f6ad9..961d96fa40577 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredPriorityQueueStateBackendMetaInfo.java @@ -51,7 +51,7 @@ public RegisteredPriorityQueueStateBackendMetaInfo(StateMetaInfoSnapshot snapsho snapshot.getName(), StateSerializerProvider.fromRestoredState( (TypeSerializerSnapshot) Preconditions.checkNotNull( - snapshot.getTypeSerializerConfigSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)))); + snapshot.getTypeSerializerSnapshot(StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER)))); Preconditions.checkState(StateMetaInfoSnapshot.BackendStateType.PRIORITY_QUEUE == snapshot.getBackendStateType()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshot.java index 1e9d9191079c5..9b05500e4d017 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshot.java @@ -81,7 +81,7 @@ public enum CommonSerializerKeys { /** The configurations of all the type serializers used with the state. */ @Nonnull - private final Map> serializerConfigSnapshots; + private final Map> serializerSnapshots; // TODO this will go away once all serializers have the restoreSerializer() factory method properly implemented. /** The serializers used by the state. */ @@ -92,8 +92,8 @@ public StateMetaInfoSnapshot( @Nonnull String name, @Nonnull BackendStateType backendStateType, @Nonnull Map options, - @Nonnull Map> serializerConfigSnapshots) { - this(name, backendStateType, options, serializerConfigSnapshots, new HashMap<>()); + @Nonnull Map> serializerSnapshots) { + this(name, backendStateType, options, serializerSnapshots, new HashMap<>()); } /** @@ -106,12 +106,12 @@ public StateMetaInfoSnapshot( @Nonnull String name, @Nonnull BackendStateType backendStateType, @Nonnull Map options, - @Nonnull Map> serializerConfigSnapshots, + @Nonnull Map> serializerSnapshots, @Nonnull Map> serializers) { this.name = name; this.backendStateType = backendStateType; this.options = options; - this.serializerConfigSnapshots = serializerConfigSnapshots; + this.serializerSnapshots = serializerSnapshots; this.serializers = serializers; } @@ -121,13 +121,13 @@ public BackendStateType getBackendStateType() { } @Nullable - public TypeSerializerSnapshot getTypeSerializerConfigSnapshot(@Nonnull String key) { - return serializerConfigSnapshots.get(key); + public TypeSerializerSnapshot getTypeSerializerSnapshot(@Nonnull String key) { + return serializerSnapshots.get(key); } @Nullable - public TypeSerializerSnapshot getTypeSerializerConfigSnapshot(@Nonnull CommonSerializerKeys key) { - return getTypeSerializerConfigSnapshot(key.toString()); + public TypeSerializerSnapshot getTypeSerializerSnapshot(@Nonnull CommonSerializerKeys key) { + return getTypeSerializerSnapshot(key.toString()); } @Nullable @@ -150,20 +150,9 @@ public String getName() { return name; } - @Nullable - public TypeSerializer restoreTypeSerializer(@Nonnull String key) { - TypeSerializerSnapshot configSnapshot = getTypeSerializerConfigSnapshot(key); - return (configSnapshot != null) ? configSnapshot.restoreSerializer() : null; - } - - @Nullable - public TypeSerializer restoreTypeSerializer(@Nonnull CommonSerializerKeys key) { - return restoreTypeSerializer(key.toString()); - } - @Nonnull - public Map> getSerializerConfigSnapshotsImmutable() { - return Collections.unmodifiableMap(serializerConfigSnapshots); + public Map> getSerializerSnapshotsImmutable() { + return Collections.unmodifiableMap(serializerSnapshots); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshotReadersWriters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshotReadersWriters.java index 4408dfcacefdb..ad1e7be287108 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshotReadersWriters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/StateMetaInfoSnapshotReadersWriters.java @@ -165,7 +165,7 @@ public void writeStateMetaInfoSnapshot( @Nonnull DataOutputView outputView) throws IOException { final Map optionsMap = snapshot.getOptionsImmutable(); final Map> serializerConfigSnapshotsMap = - snapshot.getSerializerConfigSnapshotsImmutable(); + snapshot.getSerializerSnapshotsImmutable(); outputView.writeUTF(snapshot.getName()); outputView.writeInt(snapshot.getBackendStateType().ordinal()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java index c1f08e06b3c9f..55aacb230574c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java @@ -251,7 +251,7 @@ private void assertEqualStateMetaInfoSnapshots(StateMetaInfoSnapshot expected, S Assert.assertEquals(expected.getBackendStateType(), actual.getBackendStateType()); Assert.assertEquals(expected.getOptionsImmutable(), actual.getOptionsImmutable()); Assert.assertEquals( - expected.getSerializerConfigSnapshotsImmutable(), - actual.getSerializerConfigSnapshotsImmutable()); + expected.getSerializerSnapshotsImmutable(), + actual.getSerializerSnapshotsImmutable()); } } From 388209a859e197ba862d760db2feeb1578a2c04b Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 7 Dec 2018 00:14:13 +0800 Subject: [PATCH 215/359] [FLINK-11087] [state] Incorrect broadcast state K/V serializer snapshot association when restoring from 1.5.x When restoring a broadcast state's meta information from a 1.5.x savepoint, the LegacyStateMetaInfoReaders.OperatorBackendStateMetaInfoReaderV2V3 incorrectly associates the first restored serializer as the value serializer, and the second restored serializer as the key serializer. The actual order of this should be the other way around. This bug prevents successful broadcast state restores from 1.5, both for Flink 1.6.x and 1.7.0. The commit also modifies the StatefulJobWBroadcastStateMigrationITCase to have different key / value types for its tested broadcast tests, which otherwise would not have caught this bug. --- .../metainfo/LegacyStateMetaInfoReaders.java | 33 +++++---- ...efulJobWBroadcastStateMigrationITCase.java | 66 +++++++++--------- .../_metadata | Bin 20936 -> 21096 bytes .../_metadata | Bin 20936 -> 21096 bytes .../_metadata | Bin 22318 -> 22478 bytes .../_metadata | Bin 42566 -> 42726 bytes 6 files changed, 51 insertions(+), 48 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/LegacyStateMetaInfoReaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/LegacyStateMetaInfoReaders.java index 77c267adff1d0..836edef0aac20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/LegacyStateMetaInfoReaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/metainfo/LegacyStateMetaInfoReaders.java @@ -132,11 +132,6 @@ static class OperatorBackendStateMetaInfoReaderV2V3 implements StateMetaInfoRead static final OperatorBackendStateMetaInfoReaderV2V3 INSTANCE = new OperatorBackendStateMetaInfoReaderV2V3(); - private static final String[] ORDERED_KEY_STRINGS = - new String[]{ - StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER.toString(), - StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString()}; - @Nonnull @Override public StateMetaInfoSnapshot readStateMetaInfoSnapshot( @@ -156,17 +151,25 @@ public StateMetaInfoSnapshot readStateMetaInfoSnapshot( final int listSize = stateSerializerAndConfigList.size(); StateMetaInfoSnapshot.BackendStateType stateType = listSize == 1 ? StateMetaInfoSnapshot.BackendStateType.OPERATOR : StateMetaInfoSnapshot.BackendStateType.BROADCAST; - Map> serializerConfigsMap = new HashMap<>(listSize); - for (int i = 0; i < listSize; ++i) { - Tuple2, TypeSerializerSnapshot> serializerAndConf = - stateSerializerAndConfigList.get(i); - - // this particular mapping happens to support both, V2 and V3 - String serializerKey = ORDERED_KEY_STRINGS[ORDERED_KEY_STRINGS.length - 1 - i]; - serializerConfigsMap.put( - serializerKey, - serializerAndConf.f1); + Map> serializerConfigsMap = new HashMap<>(listSize); + switch (stateType) { + case OPERATOR: + serializerConfigsMap.put( + StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString(), + stateSerializerAndConfigList.get(0).f1); + break; + case BROADCAST: + serializerConfigsMap.put( + StateMetaInfoSnapshot.CommonSerializerKeys.KEY_SERIALIZER.toString(), + stateSerializerAndConfigList.get(0).f1); + + serializerConfigsMap.put( + StateMetaInfoSnapshot.CommonSerializerKeys.VALUE_SERIALIZER.toString(), + stateSerializerAndConfigList.get(1).f1); + break; + default: + throw new IllegalStateException("Unknown operator state type " + stateType); } return new StateMetaInfoSnapshot( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobWBroadcastStateMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobWBroadcastStateMigrationITCase.java index f792362085050..3f49d84d59594 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobWBroadcastStateMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobWBroadcastStateMigrationITCase.java @@ -113,17 +113,17 @@ public void testSavepoint() throws Exception { expectedFirstState.put(2L, 2L); expectedFirstState.put(3L, 3L); - final Map expectedSecondState = new HashMap<>(); - expectedSecondState.put("0", "0"); - expectedSecondState.put("1", "1"); - expectedSecondState.put("2", "2"); - expectedSecondState.put("3", "3"); - - final Map expectedThirdState = new HashMap<>(); - expectedThirdState.put("0", "0"); - expectedThirdState.put("1", "1"); - expectedThirdState.put("2", "2"); - expectedThirdState.put("3", "3"); + final Map expectedSecondState = new HashMap<>(); + expectedSecondState.put("0", 0L); + expectedSecondState.put("1", 1L); + expectedSecondState.put("2", 2L); + expectedSecondState.put("3", 3L); + + final Map expectedThirdState = new HashMap<>(); + expectedThirdState.put(0L, "0"); + expectedThirdState.put(1L, "1"); + expectedThirdState.put(2L, "2"); + expectedThirdState.put(3L, "3"); if (executionMode == StatefulJobSavepointMigrationITCase.ExecutionMode.PERFORM_SAVEPOINT) { nonParallelSource = new MigrationTestUtils.CheckpointingNonParallelSourceWithListState(NUM_SOURCE_ELEMENTS); @@ -171,12 +171,12 @@ public Long getKey(Tuple2 value) throws Exception { "broadcast-state-1", BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO ); - final MapStateDescriptor secondBroadcastStateDesc = new MapStateDescriptor<>( - "broadcast-state-2", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + final MapStateDescriptor secondBroadcastStateDesc = new MapStateDescriptor<>( + "broadcast-state-2", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO ); - final MapStateDescriptor thirdBroadcastStateDesc = new MapStateDescriptor<>( - "broadcast-state-3", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + final MapStateDescriptor thirdBroadcastStateDesc = new MapStateDescriptor<>( + "broadcast-state-3", BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO ); BroadcastStream> npBroadcastStream = env @@ -234,7 +234,7 @@ private static class CheckpointingKeyedBroadcastFunction private MapStateDescriptor firstStateDesc; - private MapStateDescriptor secondStateDesc; + private MapStateDescriptor secondStateDesc; @Override public void open(Configuration parameters) throws Exception { @@ -245,7 +245,7 @@ public void open(Configuration parameters) throws Exception { ); secondStateDesc = new MapStateDescriptor<>( - "broadcast-state-2", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + "broadcast-state-2", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO ); } @@ -257,7 +257,7 @@ public void processElement(Tuple2 value, ReadOnlyContext ctx, Collec @Override public void processBroadcastElement(Tuple2 value, Context ctx, Collector> out) throws Exception { ctx.getBroadcastState(firstStateDesc).put(value.f0, value.f1); - ctx.getBroadcastState(secondStateDesc).put(Long.toString(value.f0), Long.toString(value.f1)); + ctx.getBroadcastState(secondStateDesc).put(Long.toString(value.f0), value.f1); } } @@ -269,14 +269,14 @@ private static class CheckpointingKeyedSingleBroadcastFunction private static final long serialVersionUID = 1333992081671604521L; - private MapStateDescriptor stateDesc; + private MapStateDescriptor stateDesc; @Override public void open(Configuration parameters) throws Exception { super.open(parameters); stateDesc = new MapStateDescriptor<>( - "broadcast-state-3", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + "broadcast-state-3", BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO ); } @@ -287,7 +287,7 @@ public void processElement(Tuple2 value, ReadOnlyContext ctx, Collec @Override public void processBroadcastElement(Tuple2 value, Context ctx, Collector> out) throws Exception { - ctx.getBroadcastState(stateDesc).put(Long.toString(value.f0), Long.toString(value.f1)); + ctx.getBroadcastState(stateDesc).put(value.f0, Long.toString(value.f1)); } } @@ -301,13 +301,13 @@ private static class CheckingKeyedBroadcastFunction private final Map expectedFirstState; - private final Map expectedSecondState; + private final Map expectedSecondState; private MapStateDescriptor firstStateDesc; - private MapStateDescriptor secondStateDesc; + private MapStateDescriptor secondStateDesc; - CheckingKeyedBroadcastFunction(Map firstState, Map secondState) { + CheckingKeyedBroadcastFunction(Map firstState, Map secondState) { this.expectedFirstState = firstState; this.expectedSecondState = secondState; } @@ -321,7 +321,7 @@ public void open(Configuration parameters) throws Exception { ); secondStateDesc = new MapStateDescriptor<>( - "broadcast-state-2", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + "broadcast-state-2", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO ); } @@ -334,8 +334,8 @@ public void processElement(Tuple2 value, ReadOnlyContext ctx, Collec } Assert.assertEquals(expectedFirstState, actualFirstState); - final Map actualSecondState = new HashMap<>(); - for (Map.Entry entry: ctx.getBroadcastState(secondStateDesc).immutableEntries()) { + final Map actualSecondState = new HashMap<>(); + for (Map.Entry entry: ctx.getBroadcastState(secondStateDesc).immutableEntries()) { actualSecondState.put(entry.getKey(), entry.getValue()); } Assert.assertEquals(expectedSecondState, actualSecondState); @@ -357,11 +357,11 @@ private static class CheckingKeyedSingleBroadcastFunction private static final long serialVersionUID = 1333992081671604521L; - private final Map expectedState; + private final Map expectedState; - private MapStateDescriptor stateDesc; + private MapStateDescriptor stateDesc; - CheckingKeyedSingleBroadcastFunction(Map state) { + CheckingKeyedSingleBroadcastFunction(Map state) { this.expectedState = state; } @@ -370,14 +370,14 @@ public void open(Configuration parameters) throws Exception { super.open(parameters); stateDesc = new MapStateDescriptor<>( - "broadcast-state-3", BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO + "broadcast-state-3", BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO ); } @Override public void processElement(Tuple2 value, ReadOnlyContext ctx, Collector> out) throws Exception { - final Map actualState = new HashMap<>(); - for (Map.Entry entry: ctx.getBroadcastState(stateDesc).immutableEntries()) { + final Map actualState = new HashMap<>(); + for (Map.Entry entry: ctx.getBroadcastState(stateDesc).immutableEntries()) { actualState.put(entry.getKey(), entry.getValue()); } Assert.assertEquals(expectedState, actualState); diff --git a/flink-tests/src/test/resources/new-stateful-broadcast-udf-migration-itcase-flink1.5-rocksdb-savepoint/_metadata b/flink-tests/src/test/resources/new-stateful-broadcast-udf-migration-itcase-flink1.5-rocksdb-savepoint/_metadata index aed61836bd9e21cee1ec1939a5a904118ea78381..2e02f1bf87daf823b3ddccaabb62dcd30ed9c16b 100644 GIT binary patch delta 1552 zcmb`HPm3Kz5WwHen1F|cJ$T3>!5oIw!xnaTRd-jT#3ho02U(ATc<6scZrPJLgg`ub z5fleMfauZ7GF~O*=0_1h&~IS9mzNL>Bm{P#hi0a``E}K=>W`m?KYtm%|7JFsf^c!r zdrUya3UJL9eB+oKTj{y2kIvp4m*2+EPv&7WcMd)D#vq*#sD%KDVgOwj10i9KeY2T9 zTX=f+;IGYQ_-eBmi;jQR(_0UQ_3t0|Dak=7ShJ)|mE`jHzBNa)FRf#Y z1)TH_RBHgX3h7-6tw+g-v|b$S`KmMwnHjp_q7knhlFI^xW2nt(2tB`wPXqkqc~$&& zt0sMqI)*2Ewv#y}Bj`c{<28XqI!h=HTwp}W$yeXji?{a7DU$V$qq<=%P$&tqYYjwF z727zm(L}yH=daJa<$j(ycAI;19(rNu8OR-VPZYg_I;wtCS6|tOrsP+DUC!zKD{W`#jgP(_*55xqAJ*$|eL8*`v}b}2 z&jcMu-2a`H)7zgeN5_Y=Q`e|kt7wIRI5MzE2kb%73zEgHtRGCv!<#pr19egz3R%T7 z8E9ujV*{xfs>Y|%oxe2DAI;$`5)2k;5UK`+lGay3Um(5-03PT=bN~PV delta 1325 zcmb7^J&R>U5Qgu$7X`%?L<2#Wk=@H^u*G&)RabX025X?ff;fSJpyOKom1?T4v8n7$ z#8hKZu0OzV0~7y&p@9a1Kft=f!p;P;%xTUSr<#7w`@YqG{;)j!e);C>Ep`^HYLKy2 z;6VjSixZLww%K}%t?lofKC|Bavi{)K8_Q}-GSP|`BS)VRxj{AQ&_lvF4M#3*wc5P7 z`{ae!ul==JEnlow>*?pe`#X2! zzPon)#$+T~f@!UN@{HyfZ^g$LJgp^~3Ux>Zo}+?Rsz5l|IuDU4#myy6zuTt3AC%i zlvYTGVhQ`7jz7A{(JgzlDM8H4z>WkenLwx;k)~9m2m1;fzlO7|H&+}+0Y-7F*z*Ef ziU+gjrm2tK$UHb7UId}6b7V4?t~*Gh38ip{B7)KswZ(BIkkgZoZ(0wbW`%~y16;*9 z61m*u9hSLC3g)@F#DQApq|jmJLtxB9R9j(41DOh{)sU{l@yFX|+o^8BXaYlrH?$yw zV#`qT7^5PotG%~(9&ENfr_gddFA|S>s!*5*)Dp=*QOhE`d#5k&Z{PiFvt{E_McFr_ z=<=rFDYBA5TFMn!3L-wL{%6_!_TH!4;l?KR;v9Q$nX7MoA$U`1AAw^qBRSvt{sTv)ElL++<+&8ZMUs=?&bO)y5$eh= z1dD@zfaucAW86w{=bs{ip#OnqUTSGPr^Bne{WF^VWEkzkk4CqjO(QOs4iDy@Z3>1Q zP>TkrD`QTUASUO%cjy@FdV1l*>5NZL*?h^GnMKqCIswBuOU=~+_@?< zQRd{ypx*F*-}(P%-nbuUE(Xci5I~wy3iK9AZVbhi;M5!5HoKm_^w~7Lv3z&^<)e@F zoP|wtV$F6=Rl9Y9oK1#`YIF3Zx~Rsx@Z)kquRW{2RhM3U^NV5q`R$Woy#iOJ!-t|h zGU)KgpyP=9Z>QzP&G(jr!%K6DVo7oW<3a*5P2)kPt8%Ui2cJ?GHS6osa_9Qh$3U%` zxrV}EvZ%?c5h#6&WD9CxRlF9>#W{g~YhJk4ii=GX(t02uUp7wPK~rBlStjRDP5T6A zu}EEcAUM%|PLrRbBy9sH7|?gy)C%XrN&j|j4#~zy34$U5CG~<9jsx$6Mdd;g75DNJ z(6u+uSp;0?0%Vzj7Lo$^+h*r^7sS&gi1!_S-ZAxk6*I#aa zGtKIgYN>e*Q`h3?QN3*R)sitTW}hR;a{u_n_3L;38kXm7-Cgdydw02a>yx<%bJr#V y%(Ru9{yaNZp|P_u=>IlK|Lid(h@RXoa#Tmn;(BWfAz(Vf~u

    P}U2aVC#%LV(m2#M_@^sJXlDNVu=J{TYZWIlD#$oc9b^20{+SxI)KLv@sD~ z7UYMQuI`AcMqes4a@Jt36(r>h%rdHLeN>Y5x8pMqv8_goR^Utrti_qT)M4nz+=O$| z^dyk~H*9a(dBda)?nNyb#au&m&UDIXZBxSBu#bf6{K{wB`PTCJ!NHAp&Eyb!^U^Ai zFCEZXnzFb7GT3oR=zNxGdiv|-@xjffE=1{*dHw79tJ}R{=hC#eqkD%Lt-lZ%nvz=E zp@Y~ykYb~5IVhdhACEpb&tc(EViVLT&Dp{oY88YK#GAu$#Og1_@r%2E=v)bEc4hOx za+BGl$ONO5TzzQ0&^|ZspXbOfyFS7IGQ9w$83!prtg*AI_mXU4E}_TC@%_WfD$$Y{ znm2QB1juOV^r}&*bC0=MY%XxbMCxTfK`q6uO%>Z1`)Cu@!n>yCM(xSE}85c@T~IJcRoH$*{9$XNQSxOnM@r! P28q=^7ml^$(_`>oZm&jO diff --git a/flink-tests/src/test/resources/new-stateful-broadcast-udf-migration-itcase-flink1.6-rocksdb-savepoint/_metadata b/flink-tests/src/test/resources/new-stateful-broadcast-udf-migration-itcase-flink1.6-rocksdb-savepoint/_metadata index 9e2b08654b28b529fa0771ae960423be71785c74..8b2a63ab41b242590169761381f35ca9d58bf0a3 100644 GIT binary patch delta 1594 zcmb`Hzl&W(6vy9vmt{dPEJ6^8(I@*@7R-^EGiQDr1g!&VHn2rNHfH9`j7?w*#Z-yJ zN(%$w{tto*N%1-h@h`Bou+Z8nwUy`XzKt=UjeBA4oqLOWzUO;B^L_Ma`1$d$bM@B8 zFq_Thy4KglhdXzcu5hXrJ;X!_$fXM0yMZWdLN?Mx((1RvYxC*P`Hd_0hS}0{lG+Ku zd8|NU8f4Qum{`PyU`iD?n~mR1_r|Zbs+N1Q`stNaq05hKKEB2cFc9I-()6jQOG`XH<8+aHc6S{;#>sO@x#0z_-A zIR}Wv#2}{oWcc5-?wwWrZ>`hq3zwEIU{<#fVz5nBiG^fBgJ3dB?@WeCRqc(_HwQb@ z=ZD+VgBM=gxXGrHwLmRuKS^)Jh=W=h3Q=fO&(Y2G=%01dC&^i|gN&Skg3pkAC5X+q zl)EvxmQ&qtzT!{x*3c_9ZXl&*Dn*uPPqFdu**{z}VJ!Bm?Yq;22z{*eJ%Rc5dIA z_TISv;vZ_pXUd`qR!D!paCcaJz4hs^`gVC|{CoOmd=5|foNxHlKBu?#kCxnAiXsd? zTh=pV3gm8v$Ytx{Dai3=QJmw$QCLv`s zFxPu~UAecoy)sxsZmEKA#sei5DA^AKR>&Fs@Vv(265*R!KxxG+Z@Mu*km=5B{E?w-Rid-1C; ztAs5{qy~~}2BK^sIcuPWV#Q&~R{ZAutM81{`gk$HczuUQSdkMqs8Iq?E(E#gd>d;D zQd}eU{|j%6b_UPqZqRGy912j-B@Owy{li0I|3Vu zsa8Q-B>_U~fTIxz(ObvSniS||1egCsFx|TL)=GRs!%kPt+*JuvHKnLTz1hw##GaCt zVmVH~?0r6aI-RWFUSHH5#v+(vHqb4x?v<-r4C>%b@g)?NXzMV%`qA`P{BC+SUOcPQ zpBG*`YtnS%!;h!m_7^K>dMMroKvroeXk|(d7Elz$Z#^(Cwq|S2kMa5`rj84F2FUjTe@qD>5we{3BrZx;d@A1Lr*TXmF)9vHC(;r87r{6z2+3zlBOh|fK zMU`LOgjDsf)Kndp>XK3XQXQV%vX(GBU0}bob7r0cMeUsh)=l);Qh-a9 zpV60GO-?o>(f8N(Y#hdZ|K4MT3N2d~yjVz4EJTKsEJ878bck9fjQ>~R=dYfbb1I27 zF{ly~5NYY7t%4BP#mcde#l_W|D+hnv&=+(?*Mf$M!2qF+5FG|6P8DMc`k0@nPbPkN zJ>9{{ET5XK@A0r?`%mGEf_BBq_<~{?Uo6tRMh(VtuuK(O!YQz_@Bg$agstQh zC|3on$jM85(}q>334=0)lS^25GY3cIs>Ya8%yOO- zNfDB4+@S*>B1t}iup%%Yqs14$I5j6OtYm!Pm}IVPsTNxR=d2Q&R4n>11 zwYep?jI~~7MiNli%b{5UNT5(fL&Z~Bym0rOssC~Blm4FE<(s|T5V0X)gBE3<l3hLag~j%d|IBfDL2-MRZXi_{{gt+b!h+q delta 1507 zcmbW1Pm3N!5XR?yH?k2HT@=Y)qJrZsh234%-PMO26Fmg7M->;W{&!tbF9|_NPI}se z0dJ9&oP)x9$tTDu`3lL+vq&z%i#2&gcViM*yP28GRL^godg}S@;qv~2<;|Vx+J*hm zQi?uQhEy#9S*ak^7Ko+U+zcDzdG+SS%gb9kpPsw17)If2T&f)~CoOQsfW$%|;-yv> zY#=)f8$V75^uud?7?w{~-(9@6Sbg;6mBr;@IJGpot!}3qmpJI!m~Xlk>&Mm~`W=%x zms%s#mJ*O+LbgVr6;mi%7Xr%ar`2UAtt=;#ZL1JeV_6#+(qL3~@nvcOQ zi~*B%w*4Q-8Sp4%Bb(UQ231@DiV`3tTfxM@9J4_ivO+%Hdj0RJVKF^A+?js4usMDF z!CSoi_q>f++!hfH5$gfifVg!?kqT6sur{BOBfX6K?H7nU{d#q0%o5m^NZ=F)NQD6o zC4-X~3X`#dehoR}%xf`y^$N(GEwNY!N&gPWRD@hp0M{#%Xpej;FN1uQ17;T-6*p9i zAvOc0cTw(~foJ29)FvX`nPk@7|M+M6?32C4lj+gkd*@E8oaj46>=DFeuX5T3nJcNf z#XF&tE16-~yfp28bA8(Or;q#f_fu~ix9arn?iVA4f~9M>bF<2Vg+Wqr5Ha_i(qv_SQ$hGTCoP_BTJ#WkIlGlMz(`VKW^(?@8JOq?o_rd@7(@&WX&n=o*=rQd8PRSxmXpxnG$mL?0s9^z5V%eIyn4( z+SKdmo&B+;>I;XcRYhmhM$`gKP&7uerm?Bw*HveJ-XFy#W%2s Date: Fri, 7 Dec 2018 00:34:46 +0800 Subject: [PATCH 216/359] [FLINK-11087] [docs] Amend compatibility table to notify issue with restoring 1.5.x braodcast state in later versions. This closes #7256. --- docs/ops/upgrading.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/ops/upgrading.md b/docs/ops/upgrading.md index 0451940723e6f..22cea4c53f0d0 100644 --- a/docs/ops/upgrading.md +++ b/docs/ops/upgrading.md @@ -274,7 +274,10 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O - + There is a known issue with resuming broadcast state created with 1.5.x in versions + 1.6.x up to 1.6.2, and 1.7.0: FLINK-11087. Users + upgrading to 1.6.x or 1.7.x series need to directly migrate to minor versions higher than 1.6.2 and 1.7.0, + respectively. 1.6.x From ce99d4012639a17f5e37488d01672cb8d281cd4d Mon Sep 17 00:00:00 2001 From: zhangzhanchun Date: Tue, 11 Dec 2018 11:00:49 +0800 Subject: [PATCH 217/359] [hotfix] [docs] Fix typo in Joining documentation This closes #7270. --- docs/dev/stream/operators/joining.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/stream/operators/joining.md b/docs/dev/stream/operators/joining.md index b95aaddbbaafc..7e9915e5e83a3 100644 --- a/docs/dev/stream/operators/joining.md +++ b/docs/dev/stream/operators/joining.md @@ -101,7 +101,7 @@ orangeStream.join(greenStream)

    ## Sliding Window Join -When performing a sliding window join, all elements with a common key and common sliding window are joined are pairwise combinations and passed on to the `JoinFunction` or `FlatJoinFunction`. Elements of one stream that do not have elements from the other stream in the current sliding window are not emitted! Note that some elements might be joined in one sliding window but not in another! +When performing a sliding window join, all elements with a common key and common sliding window are joined as pairwise combinations and passed on to the `JoinFunction` or `FlatJoinFunction`. Elements of one stream that do not have elements from the other stream in the current sliding window are not emitted! Note that some elements might be joined in one sliding window but not in another! From 04d5698b737ee98a2cf2f0da5c04bbb75a84b824 Mon Sep 17 00:00:00 2001 From: unknown Date: Thu, 6 Dec 2018 21:38:42 +0800 Subject: [PATCH 218/359] [hotfix] [docs] Fix tEnv in tableApi docs This closes #7254. --- docs/dev/table/tableApi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index b17753c16e8b4..45ee5a4e17d71 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -59,7 +59,7 @@ Table counts = orders .select("a, b.count as cnt"); // conversion to DataSet -DataSet result = tableEnv.toDataSet(counts, Row.class); +DataSet result = tEnv.toDataSet(counts, Row.class); result.print(); {% endhighlight %} From cfa1bf2a83e189238cfd122df1f852921d18d30b Mon Sep 17 00:00:00 2001 From: KarmaGYZ Date: Mon, 3 Dec 2018 19:09:28 +0800 Subject: [PATCH 219/359] [hotfix] [docs] Correct the parameter in Operators Overview doc This closes #7219. --- docs/dev/stream/operators/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/stream/operators/index.md b/docs/dev/stream/operators/index.md index 422dbbf913634..b89fa898fe794 100644 --- a/docs/dev/stream/operators/index.md +++ b/docs/dev/stream/operators/index.md @@ -434,14 +434,14 @@ IterativeStream iteration = initialStream.iterate(); DataStream iterationBody = iteration.map (/*do something*/); DataStream feedback = iterationBody.filter(new FilterFunction(){ @Override - public boolean filter(Integer value) throws Exception { + public boolean filter(Long value) throws Exception { return value > 0; } }); iteration.closeWith(feedback); DataStream output = iterationBody.filter(new FilterFunction(){ @Override - public boolean filter(Integer value) throws Exception { + public boolean filter(Long value) throws Exception { return value <= 0; } }); From a74e856b178bc41b3e4c8a1e567509573af06b5a Mon Sep 17 00:00:00 2001 From: KarmaGYZ Date: Thu, 29 Nov 2018 21:02:54 +0800 Subject: [PATCH 220/359] [FLINK-11029] [docs] Fixed incorrect parameter in Working with state doc This closes #7198. --- docs/dev/stream/state/state.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index decf1dbe9de25..a3109f2c2fa4b 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -142,7 +142,7 @@ is available in a `RichFunction` has these methods for accessing state: * `ValueState getState(ValueStateDescriptor)` * `ReducingState getReducingState(ReducingStateDescriptor)` * `ListState getListState(ListStateDescriptor)` -* `AggregatingState getAggregatingState(AggregatingState)` +* `AggregatingState getAggregatingState(AggregatingStateDescriptor)` * `FoldingState getFoldingState(FoldingStateDescriptor)` * `MapState getMapState(MapStateDescriptor)` From 4f9485f30647105d6e01717faf6fcd11ceb679fe Mon Sep 17 00:00:00 2001 From: yanghua Date: Mon, 10 Dec 2018 11:00:29 +0800 Subject: [PATCH 221/359] [FLINK-10359] [docs] Scala example in DataSet docs is broken This closes #7266. --- docs/dev/batch/dataset_transformations.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/batch/dataset_transformations.md b/docs/dev/batch/dataset_transformations.md index 96c04c9d54847..cf2da3fa91a0c 100644 --- a/docs/dev/batch/dataset_transformations.md +++ b/docs/dev/batch/dataset_transformations.md @@ -712,7 +712,7 @@ class MyCombinableGroupReducer out: Collector[String]): Unit = { val r: (String, Int) = - in.asScala.reduce( (a,b) => (a._1, a._2 + b._2) ) + in.iterator.asScala.reduce( (a,b) => (a._1, a._2 + b._2) ) // concat key and sum and emit out.collect (r._1 + "-" + r._2) } @@ -722,7 +722,7 @@ class MyCombinableGroupReducer out: Collector[(String, Int)]): Unit = { val r: (String, Int) = - in.asScala.reduce( (a,b) => (a._1, a._2 + b._2) ) + in.iterator.asScala.reduce( (a,b) => (a._1, a._2 + b._2) ) // emit tuple with key and sum out.collect(r) } From f6d93f9a8af27b64e6e7f3f1fd3b98cf4204b12d Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 18 Oct 2018 13:07:53 +0800 Subject: [PATCH 222/359] [FLINK-10252][metrics] Pass akkaFrameSize to MetricQueryService --- .../metrics/MetricRegistryConfiguration.java | 23 +++++++++++++++++-- .../runtime/metrics/MetricRegistryImpl.java | 5 +++- .../metrics/dump/MetricQueryService.java | 15 ++++++++++-- .../metrics/dump/MetricQueryServiceTest.java | 2 +- 4 files changed, 39 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java index 7188a597c8642..244a1ede5ca20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.metrics; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DelegatingConfiguration; @@ -26,6 +27,8 @@ import org.apache.flink.runtime.metrics.scope.ScopeFormats; import org.apache.flink.util.Preconditions; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,14 +69,18 @@ public class MetricRegistryConfiguration { // contains for every configured reporter its name and the configuration object private final List> reporterConfigurations; + private final long queryServiceMessageSizeLimit; + public MetricRegistryConfiguration( ScopeFormats scopeFormats, char delimiter, - List> reporterConfigurations) { + List> reporterConfigurations, + long queryServiceMessageSizeLimit) { this.scopeFormats = Preconditions.checkNotNull(scopeFormats); this.delimiter = delimiter; this.reporterConfigurations = Preconditions.checkNotNull(reporterConfigurations); + this.queryServiceMessageSizeLimit = queryServiceMessageSizeLimit; } // ------------------------------------------------------------------------ @@ -92,6 +99,10 @@ public List> getReporterConfigurations() { return reporterConfigurations; } + public long getQueryServiceMessageSizeLimit() { + return queryServiceMessageSizeLimit; + } + // ------------------------------------------------------------------------ // Static factory methods // ------------------------------------------------------------------------ @@ -160,7 +171,15 @@ public static MetricRegistryConfiguration fromConfiguration(Configuration config } } - return new MetricRegistryConfiguration(scopeFormats, delim, reporterConfigurations); + final String maxFrameSizeStr = configuration.getString(AkkaOptions.FRAMESIZE); + final String akkaConfigStr = String.format("akka {remote {netty.tcp {maximum-frame-size = %s}}}", maxFrameSizeStr); + final Config akkaConfig = ConfigFactory.parseString(akkaConfigStr); + final long maximumFrameSize = akkaConfig.getBytes("akka.remote.netty.tcp.maximum-frame-size"); + + // padding to account for serialization overhead + final long messageSizeLimitPadding = 256; + + return new MetricRegistryConfiguration(scopeFormats, delim, reporterConfigurations, maximumFrameSize - messageSizeLimitPadding); } public static MetricRegistryConfiguration defaultMetricRegistryConfiguration() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java index 6b3770907a94a..31775e242763f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java @@ -77,6 +77,8 @@ public class MetricRegistryImpl implements MetricRegistry { private final CompletableFuture terminationFuture; + private final long maximumFramesize; + @Nullable private ActorRef queryService; @@ -91,6 +93,7 @@ public class MetricRegistryImpl implements MetricRegistry { * Creates a new MetricRegistry and starts the configured reporter. */ public MetricRegistryImpl(MetricRegistryConfiguration config) { + this.maximumFramesize = config.getQueryServiceMessageSizeLimit(); this.scopeFormats = config.getScopeFormats(); this.globalDelimiter = config.getDelimiter(); this.delimiters = new ArrayList<>(10); @@ -184,7 +187,7 @@ public void startQueryService(ActorSystem actorSystem, ResourceID resourceID) { Preconditions.checkState(!isShutdown(), "The metric registry has already been shut down."); try { - queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID); + queryService = MetricQueryService.startMetricQueryService(actorSystem, resourceID, maximumFramesize); metricQueryServicePath = AkkaUtils.getAkkaURL(actorSystem, queryService); } catch (Exception e) { LOG.warn("Could not start MetricDumpActor. No metrics will be submitted to the WebInterface.", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java index 8821e0d9f4a14..ffda231f0b026 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java @@ -70,6 +70,12 @@ public String filterCharacters(String input) { private final Map> histograms = new HashMap<>(); private final Map> meters = new HashMap<>(); + private final long messageSizeLimit; + + public MetricQueryService(long messageSizeLimit) { + this.messageSizeLimit = messageSizeLimit; + } + @Override public void postStop() { serializer.close(); @@ -165,11 +171,16 @@ static String replaceInvalidChars(String str) { * @param resourceID resource ID to disambiguate the actor name * @return actor reference to the MetricQueryService */ - public static ActorRef startMetricQueryService(ActorSystem actorSystem, ResourceID resourceID) { + public static ActorRef startMetricQueryService( + ActorSystem actorSystem, + ResourceID resourceID, + long maximumFramesize) { + String actorName = resourceID == null ? METRIC_QUERY_SERVICE_NAME : METRIC_QUERY_SERVICE_NAME + "_" + resourceID.getResourceIdString(); - return actorSystem.actorOf(Props.create(MetricQueryService.class), actorName); + + return actorSystem.actorOf(Props.create(MetricQueryService.class, maximumFramesize), actorName); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java index 3767421b7d62f..afc5962c93002 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java @@ -49,7 +49,7 @@ public class MetricQueryServiceTest extends TestLogger { public void testCreateDump() throws Exception { ActorSystem s = AkkaUtils.createLocalActorSystem(new Configuration()); - ActorRef serviceActor = MetricQueryService.startMetricQueryService(s, null); + ActorRef serviceActor = MetricQueryService.startMetricQueryService(s, null, 50L); TestActorRef testActorRef = TestActorRef.create(s, Props.create(TestActor.class)); TestActor testActor = (TestActor) testActorRef.underlyingActor(); From 5c14b9526a57a820d78a80e3851d5f43e09cb99a Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 18 Oct 2018 13:07:53 +0800 Subject: [PATCH 223/359] [FLINK-10252][metrics] Handle oversized metric messages --- .../metrics/dump/MetricDumpSerialization.java | 84 +++++++++---- .../metrics/dump/MetricQueryService.java | 81 +++++++++++++ .../dump/MetricDumpSerializerTest.java | 8 +- .../metrics/dump/MetricQueryServiceTest.java | 111 +++++++++++++++++- .../legacy/metrics/MetricFetcherTest.java | 2 +- 5 files changed, 256 insertions(+), 30 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java index 16a885dd345b6..5456b56cdbd07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerialization.java @@ -73,19 +73,38 @@ public static class MetricSerializationResult implements Serializable { private static final long serialVersionUID = 6928770855951536906L; - public final byte[] serializedMetrics; + public final byte[] serializedCounters; + public final byte[] serializedGauges; + public final byte[] serializedMeters; + public final byte[] serializedHistograms; + public final int numCounters; public final int numGauges; public final int numMeters; public final int numHistograms; - public MetricSerializationResult(byte[] serializedMetrics, int numCounters, int numGauges, int numMeters, int numHistograms) { - Preconditions.checkNotNull(serializedMetrics); + public MetricSerializationResult( + byte[] serializedCounters, + byte[] serializedGauges, + byte[] serializedMeters, + byte[] serializedHistograms, + int numCounters, + int numGauges, + int numMeters, + int numHistograms) { + + Preconditions.checkNotNull(serializedCounters); + Preconditions.checkNotNull(serializedGauges); + Preconditions.checkNotNull(serializedMeters); + Preconditions.checkNotNull(serializedHistograms); Preconditions.checkArgument(numCounters >= 0); Preconditions.checkArgument(numGauges >= 0); Preconditions.checkArgument(numMeters >= 0); Preconditions.checkArgument(numHistograms >= 0); - this.serializedMetrics = serializedMetrics; + this.serializedCounters = serializedCounters; + this.serializedGauges = serializedGauges; + this.serializedMeters = serializedMeters; + this.serializedHistograms = serializedHistograms; this.numCounters = numCounters; this.numGauges = numGauges; this.numMeters = numMeters; @@ -102,7 +121,10 @@ public MetricSerializationResult(byte[] serializedMetrics, int numCounters, int */ public static class MetricDumpSerializer { - private DataOutputSerializer buffer = new DataOutputSerializer(1024 * 32); + private DataOutputSerializer countersBuffer = new DataOutputSerializer(1024 * 8); + private DataOutputSerializer gaugesBuffer = new DataOutputSerializer(1024 * 8); + private DataOutputSerializer metersBuffer = new DataOutputSerializer(1024 * 8); + private DataOutputSerializer histogramsBuffer = new DataOutputSerializer(1024 * 8); /** * Serializes the given metrics and returns the resulting byte array. @@ -126,53 +148,66 @@ public MetricSerializationResult serialize( Map> histograms, Map> meters) { - buffer.clear(); - + countersBuffer.clear(); int numCounters = 0; for (Map.Entry> entry : counters.entrySet()) { try { - serializeCounter(buffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); + serializeCounter(countersBuffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); numCounters++; } catch (Exception e) { LOG.debug("Failed to serialize counter.", e); } } + gaugesBuffer.clear(); int numGauges = 0; for (Map.Entry, Tuple2> entry : gauges.entrySet()) { try { - serializeGauge(buffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); + serializeGauge(gaugesBuffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); numGauges++; } catch (Exception e) { LOG.debug("Failed to serialize gauge.", e); } } + histogramsBuffer.clear(); int numHistograms = 0; for (Map.Entry> entry : histograms.entrySet()) { try { - serializeHistogram(buffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); + serializeHistogram(histogramsBuffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); numHistograms++; } catch (Exception e) { LOG.debug("Failed to serialize histogram.", e); } } + metersBuffer.clear(); int numMeters = 0; for (Map.Entry> entry : meters.entrySet()) { try { - serializeMeter(buffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); + serializeMeter(metersBuffer, entry.getValue().f0, entry.getValue().f1, entry.getKey()); numMeters++; } catch (Exception e) { LOG.debug("Failed to serialize meter.", e); } } - return new MetricSerializationResult(buffer.getCopyOfBuffer(), numCounters, numGauges, numMeters, numHistograms); + return new MetricSerializationResult( + countersBuffer.getCopyOfBuffer(), + gaugesBuffer.getCopyOfBuffer(), + metersBuffer.getCopyOfBuffer(), + histogramsBuffer.getCopyOfBuffer(), + numCounters, + numGauges, + numMeters, + numHistograms); } public void close() { - buffer = null; + countersBuffer = null; + gaugesBuffer = null; + metersBuffer = null; + histogramsBuffer = null; } } @@ -280,13 +315,16 @@ public static class MetricDumpDeserializer { * @return A list containing the deserialized metrics. */ public List deserialize(MetricDumpSerialization.MetricSerializationResult data) { - DataInputView in = new DataInputDeserializer(data.serializedMetrics, 0, data.serializedMetrics.length); + DataInputView countersInputView = new DataInputDeserializer(data.serializedCounters, 0, data.serializedCounters.length); + DataInputView gaugesInputView = new DataInputDeserializer(data.serializedGauges, 0, data.serializedGauges.length); + DataInputView metersInputView = new DataInputDeserializer(data.serializedMeters, 0, data.serializedMeters.length); + DataInputView histogramsInputView = new DataInputDeserializer(data.serializedHistograms, 0, data.serializedHistograms.length); - List metrics = new ArrayList<>(data.numCounters + data.numGauges + data.numHistograms + data.numMeters); + List metrics = new ArrayList<>(data.numCounters + data.numGauges + data.numMeters + data.numHistograms); for (int x = 0; x < data.numCounters; x++) { try { - metrics.add(deserializeCounter(in)); + metrics.add(deserializeCounter(countersInputView)); } catch (Exception e) { LOG.debug("Failed to deserialize counter.", e); } @@ -294,25 +332,25 @@ public List deserialize(MetricDumpSerialization.MetricSerializationR for (int x = 0; x < data.numGauges; x++) { try { - metrics.add(deserializeGauge(in)); + metrics.add(deserializeGauge(gaugesInputView)); } catch (Exception e) { LOG.debug("Failed to deserialize gauge.", e); } } - for (int x = 0; x < data.numHistograms; x++) { + for (int x = 0; x < data.numMeters; x++) { try { - metrics.add(deserializeHistogram(in)); + metrics.add(deserializeMeter(metersInputView)); } catch (Exception e) { - LOG.debug("Failed to deserialize histogram.", e); + LOG.debug("Failed to deserialize meter.", e); } } - for (int x = 0; x < data.numMeters; x++) { + for (int x = 0; x < data.numHistograms; x++) { try { - metrics.add(deserializeMeter(in)); + metrics.add(deserializeHistogram(histogramsInputView)); } catch (Exception e) { - LOG.debug("Failed to deserialize meter.", e); + LOG.debug("Failed to deserialize histogram.", e); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java index ffda231f0b026..fc69d17503d05 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/dump/MetricQueryService.java @@ -55,6 +55,7 @@ public class MetricQueryService extends UntypedActor { private static final Logger LOG = LoggerFactory.getLogger(MetricQueryService.class); public static final String METRIC_QUERY_SERVICE_NAME = "MetricQueryService"; + private static final String SIZE_EXCEEDED_LOG_TEMPLATE = "{} will not be reported as the metric dump would exceed the maximum size of {} bytes."; private static final CharacterFilter FILTER = new CharacterFilter() { @Override @@ -115,6 +116,9 @@ public void onReceive(Object message) { } } else if (message instanceof CreateDump) { MetricDumpSerialization.MetricSerializationResult dump = serializer.serialize(counters, gauges, histograms, meters); + + dump = enforceSizeLimit(dump); + getSender().tell(dump, getSelf()); } else { LOG.warn("MetricQueryServiceActor received an invalid message. " + message.toString()); @@ -125,6 +129,83 @@ public void onReceive(Object message) { } } + private MetricDumpSerialization.MetricSerializationResult enforceSizeLimit( + MetricDumpSerialization.MetricSerializationResult serializationResult) { + + int currentLength = 0; + boolean hasExceededBefore = false; + + byte[] serializedCounters = serializationResult.serializedCounters; + int numCounters = serializationResult.numCounters; + if (exceedsMessageSizeLimit(currentLength + serializationResult.serializedCounters.length)) { + logDumpSizeWouldExceedLimit("Counters", hasExceededBefore); + hasExceededBefore = true; + + serializedCounters = new byte[0]; + numCounters = 0; + } else { + currentLength += serializedCounters.length; + } + + byte[] serializedMeters = serializationResult.serializedMeters; + int numMeters = serializationResult.numMeters; + if (exceedsMessageSizeLimit(currentLength + serializationResult.serializedMeters.length)) { + logDumpSizeWouldExceedLimit("Meters", hasExceededBefore); + hasExceededBefore = true; + + serializedMeters = new byte[0]; + numMeters = 0; + } else { + currentLength += serializedMeters.length; + } + + byte[] serializedGauges = serializationResult.serializedGauges; + int numGauges = serializationResult.numGauges; + if (exceedsMessageSizeLimit(currentLength + serializationResult.serializedGauges.length)) { + logDumpSizeWouldExceedLimit("Gauges", hasExceededBefore); + hasExceededBefore = true; + + serializedGauges = new byte[0]; + numGauges = 0; + } else { + currentLength += serializedGauges.length; + } + + byte[] serializedHistograms = serializationResult.serializedHistograms; + int numHistograms = serializationResult.numHistograms; + if (exceedsMessageSizeLimit(currentLength + serializationResult.serializedHistograms.length)) { + logDumpSizeWouldExceedLimit("Histograms", hasExceededBefore); + hasExceededBefore = true; + + serializedHistograms = new byte[0]; + numHistograms = 0; + } + + return new MetricDumpSerialization.MetricSerializationResult( + serializedCounters, + serializedGauges, + serializedMeters, + serializedHistograms, + numCounters, + numGauges, + numMeters, + numHistograms); + } + + private boolean exceedsMessageSizeLimit(final int currentSize) { + return currentSize > messageSizeLimit; + } + + private void logDumpSizeWouldExceedLimit(final String metricType, boolean hasExceededBefore) { + if (LOG.isDebugEnabled()) { + LOG.debug(SIZE_EXCEEDED_LOG_TEMPLATE, metricType, messageSizeLimit); + } else { + if (!hasExceededBefore) { + LOG.info(SIZE_EXCEEDED_LOG_TEMPLATE, "Some metrics", messageSizeLimit); + } + } + } + /** * Lightweight method to replace unsupported characters. * If the string does not contain any unsupported characters, this method creates no diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java index 5f83e794ff986..1aab6f7de43ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricDumpSerializerTest.java @@ -70,7 +70,10 @@ public Object getValue() { Collections.>emptyMap()); // no metrics should be serialized - Assert.assertEquals(0, output.serializedMetrics.length); + Assert.assertEquals(0, output.serializedCounters.length); + Assert.assertEquals(0, output.serializedGauges.length); + Assert.assertEquals(0, output.serializedHistograms.length); + Assert.assertEquals(0, output.serializedMeters.length); List deserialized = deserializer.deserialize(output); Assert.assertEquals(0, deserialized.size()); @@ -141,7 +144,8 @@ public long getCount() { gauges.put(g1, new Tuple2(new QueryScopeInfo.TaskQueryScopeInfo("jid", "vid", 2, "D"), "g1")); histograms.put(h1, new Tuple2(new QueryScopeInfo.OperatorQueryScopeInfo("jid", "vid", 2, "opname", "E"), "h1")); - MetricDumpSerialization.MetricSerializationResult serialized = serializer.serialize(counters, gauges, histograms, meters); + MetricDumpSerialization.MetricSerializationResult serialized = serializer.serialize( + counters, gauges, histograms, meters); List deserialized = deserializer.deserialize(serialized); // ===== Counters ============================================================================================== diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java index afc5962c93002..ccc2236fa8499 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/dump/MetricQueryServiceTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.metrics.dump; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; @@ -38,6 +39,10 @@ import akka.testkit.TestActorRef; import org.junit.Test; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -47,9 +52,8 @@ public class MetricQueryServiceTest extends TestLogger { @Test public void testCreateDump() throws Exception { - ActorSystem s = AkkaUtils.createLocalActorSystem(new Configuration()); - ActorRef serviceActor = MetricQueryService.startMetricQueryService(s, null, 50L); + ActorRef serviceActor = MetricQueryService.startMetricQueryService(s, null, Long.MAX_VALUE); TestActorRef testActorRef = TestActorRef.create(s, Props.create(TestActor.class)); TestActor testActor = (TestActor) testActorRef.underlyingActor(); @@ -98,7 +102,10 @@ public long getCount() { MetricDumpSerialization.MetricSerializationResult dump = (MetricDumpSerialization.MetricSerializationResult) testActor.message; testActor.message = null; - assertTrue(dump.serializedMetrics.length > 0); + assertTrue(dump.serializedCounters.length > 0); + assertTrue(dump.serializedGauges.length > 0); + assertTrue(dump.serializedHistograms.length > 0); + assertTrue(dump.serializedMeters.length > 0); MetricQueryService.notifyOfRemovedMetric(serviceActor, c); MetricQueryService.notifyOfRemovedMetric(serviceActor, g); @@ -114,7 +121,103 @@ public long getCount() { MetricDumpSerialization.MetricSerializationResult emptyDump = (MetricDumpSerialization.MetricSerializationResult) testActor.message; testActor.message = null; - assertEquals(0, emptyDump.serializedMetrics.length); + assertEquals(0, emptyDump.serializedCounters.length); + assertEquals(0, emptyDump.serializedGauges.length); + assertEquals(0, emptyDump.serializedHistograms.length); + assertEquals(0, emptyDump.serializedMeters.length); + + s.terminate(); + } + + @Test + public void testHandleOversizedMetricMessage() throws Exception { + ActorSystem s = AkkaUtils.createLocalActorSystem(new Configuration()); + final long sizeLimit = 200L; + ActorRef serviceActor = MetricQueryService.startMetricQueryService(s, null, sizeLimit); + TestActorRef testActorRef = TestActorRef.create(s, Props.create(TestActor.class)); + TestActor testActor = (TestActor) testActorRef.underlyingActor(); + + final Counter c = new SimpleCounter(); + final Histogram h = new TestHistogram(); + final Meter m = new Meter() { + + @Override + public void markEvent() { + } + + @Override + public void markEvent(long n) { + } + + @Override + public double getRate() { + return 5; + } + + @Override + public long getCount() { + return 10; + } + }; + + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + final TaskManagerMetricGroup tm = new TaskManagerMetricGroup(registry, "host", "id"); + + final String gaugeValue = "Hello"; + final long requiredGaugesToExceedLimit = sizeLimit / gaugeValue.length() + 1; + List>> gauges = LongStream.range(0, requiredGaugesToExceedLimit) + .mapToObj(x -> Tuple2.of("gauge" + x, (Gauge) () -> "Hello" + x)) + .collect(Collectors.toList()); + gauges.forEach(gauge -> MetricQueryService.notifyOfAddedMetric(serviceActor, gauge.f1, gauge.f0, tm)); + + MetricQueryService.notifyOfAddedMetric(serviceActor, c, "counter", tm); + MetricQueryService.notifyOfAddedMetric(serviceActor, h, "histogram", tm); + MetricQueryService.notifyOfAddedMetric(serviceActor, m, "meter", tm); + + serviceActor.tell(MetricQueryService.getCreateDump(), testActorRef); + synchronized (testActor.lock) { + if (testActor.message == null) { + testActor.lock.wait(); + } + } + + MetricDumpSerialization.MetricSerializationResult dump = (MetricDumpSerialization.MetricSerializationResult) testActor.message; + testActor.message = null; + assertTrue(dump.serializedCounters.length > 0); + assertEquals(1, dump.numCounters); + assertTrue(dump.serializedMeters.length > 0); + assertEquals(1, dump.numMeters); + + // gauges exceeded the size limit and will be excluded + assertEquals(0, dump.serializedGauges.length); + assertEquals(0, dump.numGauges); + + assertTrue(dump.serializedHistograms.length > 0); + assertEquals(1, dump.numHistograms); + + // unregister all but one gauge to ensure gauges are reported again if the remaining fit + for (int x = 1; x < gauges.size(); x++) { + MetricQueryService.notifyOfRemovedMetric(serviceActor, gauges.get(x).f1); + } + + serviceActor.tell(MetricQueryService.getCreateDump(), testActorRef); + synchronized (testActor.lock) { + if (testActor.message == null) { + testActor.lock.wait(); + } + } + + MetricDumpSerialization.MetricSerializationResult recoveredDump = (MetricDumpSerialization.MetricSerializationResult) testActor.message; + testActor.message = null; + + assertTrue(recoveredDump.serializedCounters.length > 0); + assertEquals(1, recoveredDump.numCounters); + assertTrue(recoveredDump.serializedMeters.length > 0); + assertEquals(1, recoveredDump.numMeters); + assertTrue(recoveredDump.serializedGauges.length > 0); + assertEquals(1, recoveredDump.numGauges); + assertTrue(recoveredDump.serializedHistograms.length > 0); + assertEquals(1, recoveredDump.numHistograms); s.terminate(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java index da8182a8edea5..61c028ffab531 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/metrics/MetricFetcherTest.java @@ -99,7 +99,7 @@ public void testUpdate() throws Exception { MetricDumpSerialization.MetricSerializationResult requestMetricsAnswer = createRequestDumpAnswer(tmRID, jobID); when(jmQueryService.queryMetrics(any(Time.class))) - .thenReturn(CompletableFuture.completedFuture(new MetricDumpSerialization.MetricSerializationResult(new byte[0], 0, 0, 0, 0))); + .thenReturn(CompletableFuture.completedFuture(new MetricDumpSerialization.MetricSerializationResult(new byte[0], new byte[0], new byte[0], new byte[0], 0, 0, 0, 0))); when(tmQueryService.queryMetrics(any(Time.class))) .thenReturn(CompletableFuture.completedFuture(requestMetricsAnswer)); From 1fb8afb6cef303b018795560f91fc3dda26417c8 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 5 Dec 2018 15:46:05 +0100 Subject: [PATCH 224/359] [FLINK-11080][ES] Rework shade-plugin filters --- .../flink-connector-elasticsearch6/pom.xml | 58 ++++++++++++++----- 1 file changed, 43 insertions(+), 15 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index d1eb3d8b3d1ee..9fca691d0acac 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -197,29 +197,57 @@ under the License. true sql-jar + + + *:* + + + + com.carrotsearch:hppc + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + org.elasticsearch:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + - *:* - - + org.elasticsearch:elasticsearch - com/carrotsearch/** - com/sun/** - com/tdunning/** config/** + modules.txt + plugins.txt + org/joda/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + forbidden/** - joptsimple/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** META-INF/services/com.fasterxml.** META-INF/services/org.apache.lucene.** META-INF/services/org.elasticsearch.** - META-INF/versions/** - modules.txt - mozilla/** - org/HdrHistogram/** - org/joda/** - org/tartarus/** - org/yaml/** - plugins.txt From 51085e7f09d461321c0442b991defd148cdca95e Mon Sep 17 00:00:00 2001 From: zhangzhanchun Date: Tue, 11 Dec 2018 20:43:54 +0800 Subject: [PATCH 225/359] [hotfix] [docs] Fix typo in dynamic tables documentation This closes #7275. --- docs/dev/table/streaming/dynamic_tables.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/streaming/dynamic_tables.md b/docs/dev/table/streaming/dynamic_tables.md index 698cd673185eb..ada75a306814a 100644 --- a/docs/dev/table/streaming/dynamic_tables.md +++ b/docs/dev/table/streaming/dynamic_tables.md @@ -66,7 +66,7 @@ With these points in mind, we introduce following concept of *Dynamic tables* in Dynamic Tables & Continuous Queries --------------------------------------- -*Dynamic tables* are the core concept of Flink's Table API and SQL support for streaming data. In contrast to the static tables that represent batch data, dynamic table are changing over time. They can be queried like static batch tables. Querying a dynamic table yields a *Continuous Query*. A continuous query never terminates and produces a dynamic table as result. The query continuously updates its (dynamic) result table to reflect the changes on its input (dynamic) table. Essentially, a continuous query on a dynamic table is very similar to the definition query of a materialized view. +*Dynamic tables* are the core concept of Flink's Table API and SQL support for streaming data. In contrast to the static tables that represent batch data, dynamic tables are changing over time. They can be queried like static batch tables. Querying dynamic tables yields a *Continuous Query*. A continuous query never terminates and produces a dynamic table as result. The query continuously updates its (dynamic) result table to reflect the changes on its (dynamic) input tables. Essentially, a continuous query on a dynamic table is very similar to a query that defines a materialized view. It is important to note that the result of a continuous query is always semantically equivalent to the result of the same query being executed in batch mode on a snapshot of the input tables. @@ -177,7 +177,7 @@ When converting a dynamic table into a stream or writing it to an external syste

    -* **Upsert stream:** An upsert stream is a stream with two types of messages, *upsert messages* and *delete message*. A dynamic table that is converted into an upsert stream requires a (possibly composite) unique key. A dynamic table with unique key is converted into a dynamic table by encoding `INSERT` and `UPDATE` changes as upsert message and `DELETE` changes as delete message. The stream consuming operator needs to be aware of the unique key attribute in order to apply messages correctly. The main difference to a retract stream is that `UPDATE` changes are encoded with a single message and hence more efficient. The following figure visualizes the conversion of a dynamic table into an upsert stream. +* **Upsert stream:** An upsert stream is a stream with two types of messages, *upsert messages* and *delete messages*. A dynamic table that is converted into an upsert stream requires a (possibly composite) unique key. A dynamic table with unique key is converted into a dynamic table by encoding `INSERT` and `UPDATE` changes as upsert messages and `DELETE` changes as delete messages. The stream consuming operator needs to be aware of the unique key attribute in order to apply messages correctly. The main difference to a retract stream is that `UPDATE` changes are encoded with a single message and hence more efficient. The following figure visualizes the conversion of a dynamic table into an upsert stream.
    Dynamic tables From 4ab682bd76ecaa5fef67b077880665b5d86b03c2 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 7 Dec 2018 14:11:49 +0100 Subject: [PATCH 226/359] [FLINK-11085][build] Fix inheritance of shading filters --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 60dae4131b50d..f3fa4144dc9e8 100644 --- a/pom.xml +++ b/pom.xml @@ -1484,7 +1484,8 @@ under the License. false true ${project.basedir}/target/dependency-reduced-pom.xml - + + * From ff1821a6d2f8317d0c344719b14350ac362143d9 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 10 Dec 2018 21:33:02 +0800 Subject: [PATCH 227/359] [FLINK-11136] [table] Fix the merge logic of DISTINCT aggregates. This closes #7284. --- .../codegen/AggregationCodeGenerator.scala | 17 ++++++++++++++++- .../table/runtime/stream/sql/SqlITCase.scala | 7 ++++--- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala index 566e3d7cbc59a..57cc815fee087 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala @@ -142,6 +142,21 @@ class AggregationCodeGenerator( fields.mkString(", ") } + val parametersCodeForDistinctMerge = aggFields.map { inFields => + val fields = inFields.filter(_ > -1).zipWithIndex.map { case (f, i) => + // index to constant + if (f >= physicalInputTypes.length) { + constantFields(f - physicalInputTypes.length) + } + // index to input field + else { + s"(${CodeGenUtils.boxedTypeTermForTypeInfo(physicalInputTypes(f))}) k.getField($i)" + } + } + + fields.mkString(", ") + } + // get method signatures val classes = UserDefinedFunctionUtils.typeInfoToClass(physicalInputTypes) val constantClasses = UserDefinedFunctionUtils.typeInfoToClass(constantTypes) @@ -643,7 +658,7 @@ class AggregationCodeGenerator( | (${classOf[Row].getCanonicalName}) entry.getKey(); | Long v = (Long) entry.getValue(); | if (aDistinctAcc$i.add(k, v)) { - | ${aggs(i)}.accumulate(aAcc$i, k); + | ${aggs(i)}.accumulate(aAcc$i, ${parametersCodeForDistinctMerge(i)}); | } | } | a.setField($i, aDistinctAcc$i); diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 46dde8e022502..ddc2a68754182 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -78,6 +78,7 @@ class SqlITCase extends StreamingWithStateTestBase { val sqlQuery = "SELECT c, " + " COUNT(DISTINCT b)," + + " SUM(DISTINCT b)," + " SESSION_END(rowtime, INTERVAL '0.005' SECOND) " + "FROM MyTable " + "GROUP BY SESSION(rowtime, INTERVAL '0.005' SECOND), c " @@ -87,9 +88,9 @@ class SqlITCase extends StreamingWithStateTestBase { env.execute() val expected = Seq( - "Hello World,1,1970-01-01 00:00:00.014", // window starts at [9L] till {14L} - "Hello,1,1970-01-01 00:00:00.021", // window starts at [16L] till {21L}, not merged - "Hello,3,1970-01-01 00:00:00.015" // window starts at [1L,2L], + "Hello World,1,9,1970-01-01 00:00:00.014", // window starts at [9L] till {14L} + "Hello,1,16,1970-01-01 00:00:00.021", // window starts at [16L] till {21L}, not merged + "Hello,3,6,1970-01-01 00:00:00.015" // window starts at [1L,2L], // merged with [8L,10L], by [4L], till {15L} ) assertEquals(expected.sorted, StreamITCase.testResults.sorted) From 76bbd72e9fa633be1eff6319bda91d1bc2f99ef6 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov Date: Wed, 12 Dec 2018 19:58:05 +0100 Subject: [PATCH 228/359] [hotfix] [docs] Fix typos in Table and SQL docs This closes #7297. --- docs/dev/table/sql.md | 2 +- docs/dev/table/streaming/dynamic_tables.md | 6 +++--- docs/dev/table/streaming/joins.md | 4 ++-- docs/dev/table/streaming/temporal_tables.md | 3 +-- docs/dev/table/streaming/time_attributes.md | 2 +- 5 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index 90e20065726ff..77eddb8053c87 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -22,7 +22,7 @@ specific language governing permissions and limitations under the License. --> -SQL queries are specified with the `sqlQuery()` method of the `TableEnvironment`. The method returns the result of the SQL query as a `Table`. A `Table` can be used in [subsequent SQL and Table API queries](common.html#mixing-table-api-and-sql), be [converted into a DataSet or DataStream](common.html#integration-with-datastream-and-dataset-api), or [written to a TableSink](common.html#emit-a-table)). SQL and Table API queries can seamlessly mixed and are holistically optimized and translated into a single program. +SQL queries are specified with the `sqlQuery()` method of the `TableEnvironment`. The method returns the result of the SQL query as a `Table`. A `Table` can be used in [subsequent SQL and Table API queries](common.html#mixing-table-api-and-sql), be [converted into a DataSet or DataStream](common.html#integration-with-datastream-and-dataset-api), or [written to a TableSink](common.html#emit-a-table)). SQL and Table API queries can be seamlessly mixed and are holistically optimized and translated into a single program. In order to access a table in a SQL query, it must be [registered in the TableEnvironment](common.html#register-tables-in-the-catalog). A table can be registered from a [TableSource](common.html#register-a-tablesource), [Table](common.html#register-a-table), [DataStream, or DataSet](common.html#register-a-datastream-or-dataset-as-table). Alternatively, users can also [register external catalogs in a TableEnvironment](common.html#register-an-external-catalog) to specify the location of the data sources. diff --git a/docs/dev/table/streaming/dynamic_tables.md b/docs/dev/table/streaming/dynamic_tables.md index ada75a306814a..f8bcb9405c61b 100644 --- a/docs/dev/table/streaming/dynamic_tables.md +++ b/docs/dev/table/streaming/dynamic_tables.md @@ -53,12 +53,12 @@ The following table compares traditional relational algebra and stream processin -Despite these differences, processing streams with relational queries and SQL is not impossible. Advanced relational database systems offer a feature called *Materialized Views*. A materialized view is defined as a SQL query, just like a regular virtual view. In contrast to a virtual view, a materialized view caches the result of the query such that the query does not need to be evaluated when the view is accessed. A common challenge for caching is to prevent a cache from serving outdated results. A materialized view becomes outdated when the base tables of its definition query are modified. *Eager View Maintenance* is a technique to update materialized views and updates a materialized view as soon as its base tables are updated. +Despite these differences, processing streams with relational queries and SQL is not impossible. Advanced relational database systems offer a feature called *Materialized Views*. A materialized view is defined as a SQL query, just like a regular virtual view. In contrast to a virtual view, a materialized view caches the result of the query such that the query does not need to be evaluated when the view is accessed. A common challenge for caching is to prevent a cache from serving outdated results. A materialized view becomes outdated when the base tables of its definition query are modified. *Eager View Maintenance* is a technique to update a materialized view as soon as its base tables are updated. The connection between eager view maintenance and SQL queries on streams becomes obvious if we consider the following: - A database table is the result of a *stream* of `INSERT`, `UPDATE`, and `DELETE` DML statements, often called *changelog stream*. -- A materialized view is defined as a SQL query. In order to update the view, the query is continuously processes the changelog streams of the view's base relations. +- A materialized view is defined as a SQL query. In order to update the view, the query continuously processes the changelog streams of the view's base relations. - The materialized view is the result of the streaming SQL query. With these points in mind, we introduce following concept of *Dynamic tables* in the next section. @@ -177,7 +177,7 @@ When converting a dynamic table into a stream or writing it to an external syste


    -* **Upsert stream:** An upsert stream is a stream with two types of messages, *upsert messages* and *delete messages*. A dynamic table that is converted into an upsert stream requires a (possibly composite) unique key. A dynamic table with unique key is converted into a dynamic table by encoding `INSERT` and `UPDATE` changes as upsert messages and `DELETE` changes as delete messages. The stream consuming operator needs to be aware of the unique key attribute in order to apply messages correctly. The main difference to a retract stream is that `UPDATE` changes are encoded with a single message and hence more efficient. The following figure visualizes the conversion of a dynamic table into an upsert stream. +* **Upsert stream:** An upsert stream is a stream with two types of messages, *upsert messages* and *delete messages*. A dynamic table that is converted into an upsert stream requires a (possibly composite) unique key. A dynamic table with unique key is converted into a stream by encoding `INSERT` and `UPDATE` changes as upsert messages and `DELETE` changes as delete messages. The stream consuming operator needs to be aware of the unique key attribute in order to apply messages correctly. The main difference to a retract stream is that `UPDATE` changes are encoded with a single message and hence more efficient. The following figure visualizes the conversion of a dynamic table into an upsert stream.
    Dynamic tables diff --git a/docs/dev/table/streaming/joins.md b/docs/dev/table/streaming/joins.md index f2934066c07c5..508e8c70221c3 100644 --- a/docs/dev/table/streaming/joins.md +++ b/docs/dev/table/streaming/joins.md @@ -143,7 +143,7 @@ WHERE r.currency = o.currency Each record from the probe side will be joined with the version of the build side table at the time of the correlated time attribute of the probe side record. In order to support updates (overwrites) of previous values on the build side table, the table must define a primary key. -In our example, each record from `Orders` will be joined with the version of `Rates` at time `o.rowtime`. The `currency` field has been defined as the primary key of `Rates` before and is used to connect both tables in our example. If the query were using a processing-time notion, a newly appended order would always be joined with the most recent version of `Rates` when executing the operation. +In our example, each record from `Orders` will be joined with the version of `Rates` at time `o.rowtime`. The `currency` field has been defined as the primary key of `Rates` before and is used to connect both tables in our example. If the query were using a processing-time notion, a newly appended order would always be joined with the most recent version of `Rates` when executing the operation. In contrast to [regular joins](#regular-joins), this means that if there is a new record on the build side, it will not affect the previous results of the join. This again allows Flink to limit the number of elements that must be kept in the state. @@ -199,7 +199,7 @@ By definition, it is always the current timestamp. Thus, invocations of a proces and any updates in the underlying history table will also immediately overwrite the current values. Only the latest versions (with respect to the defined primary key) of the build side records are kept in the state. -New updates will have no effect on the previously results emitted/processed records from the probe side. +Updates of the build side will have no effect on previously emitted join results. One can think about a processing-time temporal join as a simple `HashMap` that stores all of the records from the build side. When a new record from the build side has the same key as some previous record, the old value is just simply overwritten. diff --git a/docs/dev/table/streaming/temporal_tables.md b/docs/dev/table/streaming/temporal_tables.md index b45052790b5a4..4ebb4a6e8a775 100644 --- a/docs/dev/table/streaming/temporal_tables.md +++ b/docs/dev/table/streaming/temporal_tables.md @@ -114,7 +114,7 @@ Each query to `Rates(timeAttribute)` would return the state of the `Rates` for t **Note**: Currently, Flink doesn't support directly querying the temporal table functions with a constant time attribute parameter. At the moment, temporal table functions can only be used in joins. The example above was used to provide an intuition about what the function `Rates(timeAttribute)` returns. -See also the [joining page for continuous queries](joins.html) for more information about how to join with a temporal table. +See also the page about [joins for continuous queries](joins.html) for more information about how to join with a temporal table. ### Defining Temporal Table Function @@ -171,7 +171,6 @@ val ratesHistory = env .fromCollection(ratesHistoryData) .toTable(tEnv, 'r_currency, 'r_rate, 'r_proctime.proctime) -tEnv.registerTable("Orders", orders) tEnv.registerTable("RatesHistory", ratesHistory) // Create and register TemporalTableFunction. diff --git a/docs/dev/table/streaming/time_attributes.md b/docs/dev/table/streaming/time_attributes.md index 01658132fb638..27208fb768ddb 100644 --- a/docs/dev/table/streaming/time_attributes.md +++ b/docs/dev/table/streaming/time_attributes.md @@ -40,7 +40,7 @@ Introduction to Time Attributes Time-based operations such as windows in both the [Table API]({{ site.baseurl }}/dev/table/tableApi.html#group-windows) and [SQL]({{ site.baseurl }}/dev/table/sql.html#group-windows) require information about the notion of time and its origin. Therefore, tables can offer *logical time attributes* for indicating time and accessing corresponding timestamps in table programs. -Time attributes can be part of every table schema. They are defined when creating a table from a `DataStream` or are pre-defined when using a `TableSource`. Once a time attribute has been defined at the beginning, it can be referenced as a field and can used in time-based operations. +Time attributes can be part of every table schema. They are defined when creating a table from a `DataStream` or are pre-defined when using a `TableSource`. Once a time attribute has been defined at the beginning, it can be referenced as a field and can be used in time-based operations. As long as a time attribute is not modified and is simply forwarded from one part of the query to another, it remains a valid time attribute. Time attributes behave like regular timestamps and can be accessed for calculations. If a time attribute is used in a calculation, it will be materialized and becomes a regular timestamp. Regular timestamps do not cooperate with Flink's time and watermarking system and thus can not be used for time-based operations anymore. From 1797b34e7ae276b5c20a0bf553a1709ad300b8ed Mon Sep 17 00:00:00 2001 From: KarmaGYZ Date: Mon, 10 Dec 2018 18:26:21 +0800 Subject: [PATCH 229/359] [hotfix] [docs] Improve DataSet.partitionCustom() documentation. This closes #7282. --- docs/dev/batch/index.md | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/docs/dev/batch/index.md b/docs/dev/batch/index.md index d0043647227df..0a498dfe2a1f6 100644 --- a/docs/dev/batch/index.md +++ b/docs/dev/batch/index.md @@ -401,12 +401,14 @@ DataSet result = in.partitionByRange(0) Custom Partitioning -

    Manually specify a partitioning over the data. +

    Assigns records based on a key to a specific partition using a custom Partitioner function. + The key can be specified as position key, expression key, and key selector function.
    - Note: This method works only on single field keys.

    + Note: This method only works with a single field key.

    {% highlight java %} DataSet> in = // [...] -DataSet result = in.partitionCustom(Partitioner partitioner, key) +DataSet result = in.partitionCustom(partitioner, key) + .mapPartition(new PartitionMapper()); {% endhighlight %} @@ -703,13 +705,14 @@ val result = in.partitionByRange(0).mapPartition { ... } Custom Partitioning -

    Manually specify a partitioning over the data. +

    Assigns records based on a key to a specific partition using a custom Partitioner function. + The key can be specified as position key, expression key, and key selector function.
    - Note: This method works only on single field keys.

    + Note: This method only works with a single field key.

    {% highlight scala %} val in: DataSet[(Int, String)] = // [...] val result = in - .partitionCustom(partitioner: Partitioner[K], key) + .partitionCustom(partitioner, key).mapPartition { ... } {% endhighlight %} From b690a3ca265a9416d63d1856853a1a947093192d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 13 Dec 2018 13:52:02 +0100 Subject: [PATCH 230/359] [hotfix] [docs] Add notice about buggy DATE_FORMAT function --- docs/dev/table/functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/table/functions.md b/docs/dev/table/functions.md index e652c304e932c..1873d04a460ae 100644 --- a/docs/dev/table/functions.md +++ b/docs/dev/table/functions.md @@ -3487,7 +3487,7 @@ DATE_FORMAT(timestamp, string) {% endhighlight %} -

    Returns a string that formats timestamp with a specified format string. The format specification is given in the Date Format Specifier table.

    +

    Attention This function has serious bugs and should not be used for now. Please implement a custom UDF instead or use EXTRACT as a workaround.

    From e8855e7b1179d07978bd6811dc5fca1076464e67 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 13 Dec 2018 14:03:52 +0100 Subject: [PATCH 231/359] [hotfix] [docs] Add notice about buggy dateFormat() function --- docs/dev/table/functions.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/dev/table/functions.md b/docs/dev/table/functions.md index 1873d04a460ae..682f519eb3ce6 100644 --- a/docs/dev/table/functions.md +++ b/docs/dev/table/functions.md @@ -3756,8 +3756,7 @@ dateFormat(TIMESTAMP, STRING) {% endhighlight %} -

    Returns a string that formats TIMESTAMP with a specified format STRING. The format specification is given in the Date Format Specifier table.

    -

    E.g., dateFormat(ts, '%Y, %d %M') results in strings formatted as "2017, 05 May".

    +

    Attention This function has serious bugs and should not be used for now. Please implement a custom UDF instead or use extract() as a workaround.

    @@ -4014,8 +4013,7 @@ dateFormat(TIMESTAMP, STRING) {% endhighlight %} -

    Returns a string that formats TIMESTAMP with a specified format STRING. The format specification is given in the Date Format Specifier table.

    -

    E.g., dateFormat('ts, "%Y, %d %M") results in strings formatted as "2017, 05 May".

    +

    Attention This function has serious bugs and should not be used for now. Please implement a custom UDF instead or use extract() as a workaround.

    From cd4f94e0027d300e02f5f568787b2ef0ffc6be4e Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Tue, 11 Dec 2018 07:21:14 -0800 Subject: [PATCH 232/359] [FLINK-11145] Fix Hadoop version handling in binary release script --- tools/releasing/create_binary_release.sh | 28 ++++++++++++------------ 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tools/releasing/create_binary_release.sh b/tools/releasing/create_binary_release.sh index f4d7c5770ca9c..444e01b28cee9 100755 --- a/tools/releasing/create_binary_release.sh +++ b/tools/releasing/create_binary_release.sh @@ -20,8 +20,8 @@ ## ## Variables with defaults (if not overwritten by environment) ## -SCALA_VERSION=none -HADOOP_VERSION=none +SCALA_VERSION=${SCALA_VERSION:-none} +HADOOP_VERSION=${HADOOP_VERSION:-none} SKIP_GPG=${SKIP_GPG:-false} MVN=${MVN:-mvn} @@ -95,27 +95,27 @@ make_binary_release() { cd ${FLINK_DIR} } +HADOOP_CLASSIFIERS=("24" "26" "27" "28") +HADOOP_VERSIONS=("2.4.1" "2.6.5" "2.7.5" "2.8.3") + if [ "$SCALA_VERSION" == "none" ] && [ "$HADOOP_VERSION" == "none" ]; then make_binary_release "" "-DwithoutHadoop" "2.12" - make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "2.12" - make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "2.12" - make_binary_release "hadoop27" "-Dhadoop.version=2.7.5" "2.12" - make_binary_release "hadoop28" "-Dhadoop.version=2.8.3" "2.12" + for i in "${!HADOOP_CLASSIFIERS[@]}"; do + make_binary_release "hadoop${HADOOP_CLASSIFIERS[$i]}" "-Dhadoop.version=${HADOOP_VERSIONS[$i]}" "2.12" + done make_binary_release "" "-DwithoutHadoop" "2.11" - make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "2.11" - make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "2.11" - make_binary_release "hadoop27" "-Dhadoop.version=2.7.5" "2.11" - make_binary_release "hadoop28" "-Dhadoop.version=2.8.3" "2.11" + for i in "${!HADOOP_CLASSIFIERS[@]}"; do + make_binary_release "hadoop${HADOOP_CLASSIFIERS[$i]}" "-Dhadoop.version=${HADOOP_VERSIONS[$i]}" "2.11" + done elif [ "$SCALA_VERSION" == none ] && [ "$HADOOP_VERSION" != "none" ] then make_binary_release "hadoop2" "-Dhadoop.version=$HADOOP_VERSION" "2.11" elif [ "$SCALA_VERSION" != none ] && [ "$HADOOP_VERSION" == "none" ] then make_binary_release "" "-DwithoutHadoop" "$SCALA_VERSION" - make_binary_release "hadoop24" "-Dhadoop.version=2.4.1" "$SCALA_VERSION" - make_binary_release "hadoop26" "-Dhadoop.version=2.6.5" "$SCALA_VERSION" - make_binary_release "hadoop27" "-Dhadoop.version=2.7.3" "$SCALA_VERSION" - make_binary_release "hadoop28" "-Dhadoop.version=2.8.0" "$SCALA_VERSION" + for i in "${!HADOOP_CLASSIFIERS[@]}"; do + make_binary_release "hadoop${HADOOP_CLASSIFIERS[$i]}" "-Dhadoop.version=${HADOOP_VERSIONS[$i]}" "$SCALA_VERSION" + done else make_binary_release "hadoop2x" "-Dhadoop.version=$HADOOP_VERSION" "$SCALA_VERSION" fi From 43bb2a927ad42909fea991b438b6158a605f1fe9 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 11 Dec 2018 14:52:54 +0100 Subject: [PATCH 233/359] [FLINK-10566] Fix exponential planning time of large programs The traversal of the DAG is not efficient enough at some places which can lead to very long plan creation times. This introduces caching for the traversal to avoid traversing nodes multiple times. Caching is performed at two places: - when registering Kryo types - when determining the maximum parallelism --- .../org/apache/flink/api/common/Plan.java | 7 +- .../flink/api/java/ExecutionEnvironment.java | 9 ++- .../flink/test/planning/LargePlanTest.java | 81 +++++++++++++++++++ 3 files changed, 94 insertions(+), 3 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java index efbc4fac03906..32eed69e2cee0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/Plan.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/Plan.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map.Entry; import java.util.Set; @@ -361,10 +362,14 @@ public int getMaximumParallelism() { private static final class MaxDopVisitor implements Visitor> { + private final Set visitedOperators = new HashSet<>(); private int maxDop = -1; - + @Override public boolean preVisit(Operator visitable) { + if (!visitedOperators.add(visitable)) { + return false; + } this.maxDop = Math.max(this.maxDop, visitable.getParallelism()); return true; } 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 2bed411840df0..e6a2747e8a96a 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 @@ -73,6 +73,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Set; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -963,12 +964,16 @@ public Plan createProgramPlan(String jobName, boolean clearSinks) { if (!config.isAutoTypeRegistrationDisabled()) { plan.accept(new Visitor>() { - private final HashSet> deduplicator = new HashSet<>(); + private final Set> registeredTypes = new HashSet<>(); + private final Set> visitedOperators = new HashSet<>(); @Override public boolean preVisit(org.apache.flink.api.common.operators.Operator visitable) { + if (!visitedOperators.add(visitable)) { + return false; + } OperatorInformation opInfo = visitable.getOperatorInfo(); - Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, deduplicator); + Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes); return true; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.java new file mode 100644 index 0000000000000..6c30af88ad49e --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/planning/LargePlanTest.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.test.planning; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.OptimizerPlanEnvironment; +import org.apache.flink.client.program.PreviewPlanEnvironment; + +import org.junit.Test; + +/** + * Tests that large programs can be compiled to a Plan in reasonable amount of time. + */ +public class LargePlanTest { + + @Test(expected = OptimizerPlanEnvironment.ProgramAbortException.class, timeout = 15_000) + public void testPlanningOfLargePlan() throws Exception { + runProgram(new PreviewPlanEnvironment(), 10, 50); + } + + private static void runProgram(ExecutionEnvironment env, int depth, int width) throws Exception { + DataSet input = env.fromElements("a", "b", "c"); + DataSet stats = null; + + for (int i = 0; i < depth; i++) { + stats = analyze(input, stats, width / (i + 1) + 1); + } + + stats.output(new DiscardingOutputFormat<>()); + env.execute("depth " + depth + " width " + width); + } + + private static DataSet analyze(DataSet input, DataSet stats, int branches) { + for (int i = 0; i < branches; i++) { + final int ii = i; + + if (stats != null) { + input = input.map( + new RichMapFunction() { + @Override + public String map(String value) { + return value; + } + }).withBroadcastSet(stats.map(s -> "(" + s + ").map"), "stats"); + } + + DataSet branch = input + .map(s -> new Tuple2<>(0, s + ii)).returns(Types.TUPLE(Types.STRING, Types.INT)) + .groupBy(0) + .minBy(1) + .map(kv -> kv.f1).returns(Types.STRING); + if (stats == null) { + stats = branch; + } else { + stats = stats.union(branch); + } + } + return stats.map(s -> "(" + s + ").stats"); + } +} From 11644a79a759597e70e0e5f23edeab3aa1cc7bbb Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Fri, 14 Dec 2018 13:29:48 +0800 Subject: [PATCH 234/359] [hotfix] [docs] Improve the correctness in Detecting Patterns doc This closes #7307. --- docs/dev/table/streaming/match_recognize.md | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docs/dev/table/streaming/match_recognize.md b/docs/dev/table/streaming/match_recognize.md index 3cd1ed09cc9a2..3c805cf013333 100644 --- a/docs/dev/table/streaming/match_recognize.md +++ b/docs/dev/table/streaming/match_recognize.md @@ -107,7 +107,7 @@ The table has a following schema: {% highlight text %} Ticker - |-- symbol: Long # symbol of the stock + |-- symbol: String # symbol of the stock |-- price: Long # price of the stock |-- tax: Long # tax liability of the stock |-- rowtime: TimeIndicatorTypeInfo(rowtime) # point in time when the change to those values happened @@ -256,13 +256,13 @@ FROM Ticker ORDER BY rowtime MEASURES C.price AS lastPrice - PATTERN (A B* C) ONE ROW PER MATCH AFTER MATCH SKIP PAST LAST ROW + PATTERN (A B* C) DEFINE A AS A.price > 10, B AS B.price < 15, - C AS B.price > 12 + C AS C.price > 12 ) {% endhighlight %} @@ -293,6 +293,7 @@ The same query where `B*` is modified to `B*?`, which means that `B*` should be symbol lastPrice ======== =========== XYZ 13 + XYZ 16 {% endhighlight %} The pattern variable `B` matches only to the row with price `12` instead of swallowing the rows with prices `12`, `13`, and `14`. @@ -334,8 +335,8 @@ FROM Ticker FIRST(A.price) AS startPrice, LAST(A.price) AS topPrice, B.price AS lastPrice - PATTERN (A+ B) ONE ROW PER MATCH + PATTERN (A+ B) DEFINE A AS LAST(A.price, 1) IS NULL OR A.price > LAST(A.price, 1), B AS B.price < LAST(A.price) @@ -704,9 +705,9 @@ FROM Ticker SUM(A.price) AS sumPrice, FIRST(rowtime) AS startTime, LAST(rowtime) AS endTime - PATTERN (A+ C) ONE ROW PER MATCH [AFTER MATCH STRATEGY] + PATTERN (A+ C) DEFINE A AS SUM(A.price) < 30 ) @@ -777,7 +778,7 @@ The last result matched against the rows #5, #6. This combination will produce a runtime exception because one would always try to start a new match where the last one started. This would produce an infinite loop and, thus, is prohibited. -One has to keep in mind that in case of the `SKIP TO FIRST/LAST variable`strategy it might be possible that there are no rows mapped to that +One has to keep in mind that in case of the `SKIP TO FIRST/LAST variable` strategy it might be possible that there are no rows mapped to that variable (e.g. for pattern `A*`). In such cases, a runtime exception will be thrown as the standard requires a valid row to continue the matching. From 426301328e59ce67e2ab59702ab24f2076083a39 Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Fri, 14 Dec 2018 19:59:20 +0800 Subject: [PATCH 235/359] [hotfix] [docs] Correct the field name in Connect to External Systems doc This closes #7311. --- docs/dev/table/connect.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index d8677714fa86f..e7f9263c29d60 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -154,7 +154,7 @@ tableEnvironment new Schema() .field("rowtime", Types.SQL_TIMESTAMP) .rowtime(new Rowtime() - .timestampsFromField("ts") + .timestampsFromField("timestamp") .watermarksPeriodicBounded(60000) ) .field("user", Types.LONG) @@ -1166,7 +1166,7 @@ ClusterBuilder builder = ... // configure Cassandra cluster connection CassandraAppendTableSink sink = new CassandraAppendTableSink( builder, // the query must match the schema of the table - INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?)); + "INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?)"); tableEnv.registerTableSink( "cassandraOutputTable", @@ -1187,7 +1187,7 @@ val builder: ClusterBuilder = ... // configure Cassandra cluster connection val sink: CassandraAppendTableSink = new CassandraAppendTableSink( builder, // the query must match the schema of the table - INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?)) + "INSERT INTO flink.myTable (id, name, value) VALUES (?, ?, ?)") tableEnv.registerTableSink( "cassandraOutputTable", From d00dd3323d53cba323bde5ebfe390ffdb4d777d4 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 14 Dec 2018 17:10:09 +0100 Subject: [PATCH 236/359] Revert "[FLINK-9555][scala-shell] Support table api in scala shell." This reverts commit efc73a872ac52e314bb1a05b9c5ed045cde6df1f. --- flink-scala-shell/pom.xml | 7 --- .../apache/flink/api/scala/FlinkILoop.scala | 39 +++---------- .../flink/api/scala/ScalaShellITCase.scala | 55 ------------------- .../start-script/start-scala-shell.sh | 7 --- 4 files changed, 9 insertions(+), 99 deletions(-) diff --git a/flink-scala-shell/pom.xml b/flink-scala-shell/pom.xml index dfb7416b36104..5a96804794590 100644 --- a/flink-scala-shell/pom.xml +++ b/flink-scala-shell/pom.xml @@ -78,13 +78,6 @@ under the License. ${scala.version} - - org.apache.flink - flink-table_${scala.binary.version} - ${project.version} - provided - - diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala index c124d8ea8a3c7..4b6e886994abc 100644 --- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala +++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkILoop.scala @@ -23,8 +23,6 @@ import java.io.{BufferedReader, File, FileOutputStream} import org.apache.flink.api.java.{JarHelper, ScalaShellRemoteEnvironment, ScalaShellRemoteStreamEnvironment} import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.configuration.Configuration -import org.apache.flink.table.api.TableEnvironment -import org.apache.flink.table.api.scala.{BatchTableEnvironment, StreamTableEnvironment} import org.apache.flink.util.AbstractID import scala.tools.nsc.interpreter._ @@ -92,17 +90,10 @@ class FlinkILoop( } // local environment - val ( - scalaBenv: ExecutionEnvironment, - scalaSenv: StreamExecutionEnvironment, - scalaBTEnv: BatchTableEnvironment, - scalaSTEnv: StreamTableEnvironment - ) = { + val (scalaBenv: ExecutionEnvironment, scalaSenv: StreamExecutionEnvironment) = { val scalaBenv = new ExecutionEnvironment(remoteBenv) val scalaSenv = new StreamExecutionEnvironment(remoteSenv) - val scalaBTEnv = TableEnvironment.getTableEnvironment(scalaBenv) - val scalaSTEnv = TableEnvironment.getTableEnvironment(scalaSenv) - (scalaBenv,scalaSenv,scalaBTEnv,scalaSTEnv) + (scalaBenv,scalaSenv) } /** @@ -148,10 +139,7 @@ class FlinkILoop( "org.apache.flink.api.scala._", "org.apache.flink.api.scala.utils._", "org.apache.flink.streaming.api.scala._", - "org.apache.flink.streaming.api.windowing.time._", - "org.apache.flink.table.api._", - "org.apache.flink.table.api.scala._", - "org.apache.flink.types.Row" + "org.apache.flink.streaming.api.windowing.time._" ) override def createInterpreter(): Unit = { @@ -164,8 +152,6 @@ class FlinkILoop( // set execution environment intp.bind("benv", this.scalaBenv) intp.bind("senv", this.scalaSenv) - intp.bind("btenv", this.scalaBTEnv) - intp.bind("stenv", this.scalaSTEnv) } } @@ -257,29 +243,22 @@ class FlinkILoop( F L I N K - S C A L A - S H E L L -NOTE: Use the prebound Execution Environments and Table Environment to implement batch or streaming programs. +NOTE: Use the prebound Execution Environments to implement batch or streaming programs. - Batch - Use the 'benv' and 'btenv' variable + Batch - Use the 'benv' variable * val dataSet = benv.readTextFile("/path/to/data") * dataSet.writeAsText("/path/to/output") * benv.execute("My batch program") - * - * val batchTable = btenv.fromDataSet(dataSet) - * btenv.registerTable("tableName", batchTable) - * val result = btenv.sqlQuery("SELECT * FROM tableName").collect - HINT: You can use print() on a DataSet to print the contents or collect() - a sql query result back to the shell. - Streaming - Use the 'senv' and 'stenv' variable + HINT: You can use print() on a DataSet to print the contents to the shell. + + Streaming - Use the 'senv' variable * val dataStream = senv.fromElements(1, 2, 3, 4) * dataStream.countWindowAll(2).sum(0).print() - * - * val streamTable = stenv.fromDataStream(dataStream, 'num) - * val resultTable = streamTable.select('num).where('num % 2 === 1 ) - * resultTable.toAppendStream[Row].print() * senv.execute("My streaming program") + HINT: You can only print a DataStream to the shell in local mode. """ // scalastyle:on diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala index fc90d8d143c60..337e4fb9be9e6 100644 --- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala +++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala @@ -168,61 +168,6 @@ class ScalaShellITCase extends TestLogger { Assert.assertTrue(output.contains("WC(world,10)")) } - @Test - def testSimpleSelectWithFilterBatchTableAPIQuery: Unit = { - val input = - """ - |val data = Seq( - | (1, 1L, "Hi"), - | (2, 2L, "Hello"), - | (3, 2L, "Hello world")) - |val t = benv.fromCollection(data).toTable(btenv, 'a, 'b, 'c).select('a,'c).where( - |'a% 2 === 1 ) - |val results = t.toDataSet[Row].collect() - |results.foreach(println) - |:q - """.stripMargin - val output = processInShell(input) - Assert.assertFalse(output.contains("failed")) - Assert.assertFalse(output.contains("error")) - Assert.assertFalse(output.contains("Exception")) - Assert.assertTrue(output.contains("1,Hi")) - Assert.assertTrue(output.contains("3,Hello world")) - } - - @Test - def testGroupedAggregationStreamTableAPIQuery: Unit = { - val input = - """ - | val data = List( - | ("Hello", 1), - | ("word", 1), - | ("Hello", 1), - | ("bark", 1), - | ("bark", 1), - | ("bark", 1), - | ("bark", 1), - | ("bark", 1), - | ("bark", 1), - | ("flink", 1) - | ) - | val stream = senv.fromCollection(data) - | val table = stream.toTable(stenv, 'word, 'num) - | val resultTable = table.groupBy('word).select('num.sum as 'count).groupBy('count).select( - | 'count,'count.count as 'frequency) - | val results = resultTable.toRetractStream[Row] - | results.print - | senv.execute - """.stripMargin - val output = processInShell(input) - Assert.assertTrue(output.contains("6,1")) - Assert.assertTrue(output.contains("1,2")) - Assert.assertTrue(output.contains("2,1")) - Assert.assertFalse(output.contains("failed")) - Assert.assertFalse(output.contains("error")) - Assert.assertFalse(output.contains("Exception")) - } - /** * Submit external library. * Disabled due to FLINK-7111. diff --git a/flink-scala-shell/start-script/start-scala-shell.sh b/flink-scala-shell/start-script/start-scala-shell.sh index e3571145f13ed..033d505032229 100644 --- a/flink-scala-shell/start-script/start-scala-shell.sh +++ b/flink-scala-shell/start-script/start-scala-shell.sh @@ -52,13 +52,6 @@ bin=`cd "$bin"; pwd` FLINK_CLASSPATH=`constructFlinkClassPath` -# Append flink-table jar into class path -opt=`dirname "$0"` -opt=`cd ../"$opt"/opt; pwd` -FLINK_TABLE_LIB_PATH=$opt/`ls $opt|grep flink-table_*` -FLINK_CLASSPATH=$FLINK_CLASSPATH:$FLINK_TABLE_LIB_PATH - - # https://issues.scala-lang.org/browse/SI-6502, cant load external jars interactively # in scala shell since 2.10, has to be done at startup # checks arguments for additional classpath and adds it to the "standard classpath" From 5ce9f228959469b9e7c8e585aaffadc7eff5d028 Mon Sep 17 00:00:00 2001 From: blueszheng Date: Mon, 10 Dec 2018 22:04:05 +0800 Subject: [PATCH 237/359] [FLINK-11083] [Table&SQL] CRowSerializerConfigSnapshot is not instantiable This closes #7267. --- .../table/runtime/types/CRowSerializer.scala | 8 +- .../runtime/types/CRowSerializerTest.scala | 80 ++++++++++++++++++- 2 files changed, 84 insertions(+), 4 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala index 0ce3aee3739e7..b3fe508515152 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala @@ -81,7 +81,7 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali // -------------------------------------------------------------------------------------------- override def snapshotConfiguration(): TypeSerializerConfigSnapshot[CRow] = { - new CRowSerializer.CRowSerializerConfigSnapshot(rowSerializer) + new CRowSerializer.CRowSerializerConfigSnapshot(Array(rowSerializer)) } override def ensureCompatibility( @@ -115,9 +115,13 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali object CRowSerializer { - class CRowSerializerConfigSnapshot(rowSerializers: TypeSerializer[Row]*) + class CRowSerializerConfigSnapshot(rowSerializers: Array[TypeSerializer[Row]]) extends CompositeTypeSerializerConfigSnapshot[CRow](rowSerializers: _*) { + def this() { + this(Array.empty) + } + override def getVersion: Int = CRowSerializerConfigSnapshot.VERSION } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowSerializerTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowSerializerTest.scala index 7483b04d9cac1..055501a6a0160 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowSerializerTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/types/CRowSerializerTest.scala @@ -18,8 +18,20 @@ package org.apache.flink.table.runtime.types -import org.apache.flink.util.TestLogger -import org.junit.Test +import org.apache.flink.api.common.state.{ListState, ListStateDescriptor} +import org.apache.flink.api.common.typeinfo.Types +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend +import org.apache.flink.streaming.api.functions.KeyedProcessFunction +import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, KeyedProcessOperator} +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness +import org.apache.flink.types.Row +import org.apache.flink.util.{Collector, InstantiationUtil, TestLogger} + +import org.junit.{Assert, Test} class CRowSerializerTest extends TestLogger { @@ -29,6 +41,70 @@ class CRowSerializerTest extends TestLogger { @Test def testDefaultConstructor(): Unit = { new CRowSerializer.CRowSerializerConfigSnapshot() + + InstantiationUtil.instantiate(classOf[CRowSerializer.CRowSerializerConfigSnapshot]) + } + + @Test + def testStateRestore(): Unit = { + + class IKeyedProcessFunction extends KeyedProcessFunction[Integer, Integer, Integer] { + var state: ListState[CRow] = _ + override def open(parameters: Configuration): Unit = { + val stateDesc = new ListStateDescriptor[CRow]("CRow", + new CRowTypeInfo(new RowTypeInfo(Types.INT))) + state = getRuntimeContext.getListState(stateDesc) + } + override def processElement(value: Integer, + ctx: KeyedProcessFunction[Integer, Integer, Integer]#Context, + out: Collector[Integer]): Unit = { + state.add(new CRow(Row.of(value), true)) + } + } + + val operator = new KeyedProcessOperator[Integer, Integer, Integer](new IKeyedProcessFunction) + + var testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, Integer, Integer]( + operator, + new KeySelector[Integer, Integer] { + override def getKey(value: Integer): Integer= -1 + }, + Types.INT, 1, 1, 0) + testHarness.setup() + testHarness.open() + + testHarness.processElement(new StreamRecord[Integer](1, 1L)) + testHarness.processElement(new StreamRecord[Integer](2, 1L)) + testHarness.processElement(new StreamRecord[Integer](3, 1L)) + + Assert.assertEquals(1, numKeyedStateEntries(operator)) + + val snapshot = testHarness.snapshot(0L, 0L) + testHarness.close() + + testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, Integer, Integer]( + operator, + new KeySelector[Integer, Integer] { + override def getKey(value: Integer): Integer= -1 + }, + Types.INT, 1, 1, 0) + testHarness.setup() + + testHarness.initializeState(snapshot) + + testHarness.open() + + Assert.assertEquals(1, numKeyedStateEntries(operator)) + + testHarness.close() + } + + def numKeyedStateEntries(operator: AbstractStreamOperator[_]): Int = { + val keyedStateBackend = operator.getKeyedStateBackend + keyedStateBackend match { + case hksb: HeapKeyedStateBackend[_] => hksb.numKeyValueStateEntries + case _ => throw new UnsupportedOperationException + } } } From bfa73ae7523bab17c95a5d78f993cbb5ad4e3e8f Mon Sep 17 00:00:00 2001 From: Jihyun Cho Date: Fri, 28 Sep 2018 10:00:10 +0200 Subject: [PATCH 238/359] [FLINK-10457][fs-connector] Add SequenceFile support to StreamingFileSink. --- .../avro/ParquetStreamingFileSinkITCase.java | 2 +- flink-formats/flink-sequencefile/pom.xml | 107 +++++++++++ .../sequencefile/SequenceFileWriter.java | 61 +++++++ .../SequenceFileWriterFactory.java | 168 ++++++++++++++++++ .../sequencefile/SequenceFileSinkITCase.java | 117 ++++++++++++ .../src/test/resources/log4j-test.properties | 23 +++ flink-formats/pom.xml | 1 + .../flink/test/util}/FiniteTestSource.java | 2 +- 8 files changed, 479 insertions(+), 2 deletions(-) create mode 100644 flink-formats/flink-sequencefile/pom.xml create mode 100644 flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java create mode 100644 flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java create mode 100644 flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.java create mode 100644 flink-formats/flink-sequencefile/src/test/resources/log4j-test.properties rename {flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/testutils => flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util}/FiniteTestSource.java (97%) diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java index d1f0a5f57787f..540f762c6419d 100644 --- a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/ParquetStreamingFileSinkITCase.java @@ -22,11 +22,11 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; import org.apache.flink.formats.parquet.generated.Address; -import org.apache.flink.formats.parquet.testutils.FiniteTestSource; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.test.util.FiniteTestSource; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; diff --git a/flink-formats/flink-sequencefile/pom.xml b/flink-formats/flink-sequencefile/pom.xml new file mode 100644 index 0000000000000..0c5ade4575d3e --- /dev/null +++ b/flink-formats/flink-sequencefile/pom.xml @@ -0,0 +1,107 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-formats + 1.7-SNAPSHOT + .. + + + flink-sequencefile + flink-sequencefile + + jar + + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + + org.apache.flink + flink-shaded-hadoop2 + ${project.version} + provided + + + org.apache.flink + flink-hadoop-fs + ${project.version} + provided + + + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-hadoop-compatibility_${scala.binary.version} + ${project.version} + test + + + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + + enforce + + + true + + + + + + + diff --git a/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java b/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java new file mode 100644 index 0000000000000..fa8fab240937e --- /dev/null +++ b/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java @@ -0,0 +1,61 @@ +/* + * 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.formats.sequencefile; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Writable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link BulkWriter} implementation that wraps a {@link SequenceFile.Writer}. + * + * @param The type of key written. + * @param The type of value written. + */ +@PublicEvolving +public class SequenceFileWriter implements BulkWriter> { + private final SequenceFile.Writer writer; + + public SequenceFileWriter(SequenceFile.Writer writer) { + this.writer = checkNotNull(writer); + } + + @Override + public void addElement(Tuple2 element) throws IOException { + writer.append(element.f0, element.f1); + } + + @Override + public void flush() throws IOException { + writer.hsync(); + } + + @Override + public void finish() throws IOException { + writer.close(); + } +} + diff --git a/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java b/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java new file mode 100644 index 0000000000000..90a87e77bcabe --- /dev/null +++ b/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.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.formats.sequencefile; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A factory that creates a SequenceFile {@link BulkWriter}. + * + * @param The type of key to write. It should be writable. + * @param The type of value to write. It should be writable. + */ +@PublicEvolving +public class SequenceFileWriterFactory implements BulkWriter.Factory> { + private static final long serialVersionUID = 1L; + + private final SerializableHadoopConfiguration serdeHadoopConfig; + private final Class keyClass; + private final Class valueClass; + private final String compressionCodecName; + private final SequenceFile.CompressionType compressionType; + + /** + * Creates a new SequenceFileWriterFactory using the given builder to assemble the + * SequenceFileWriter. + * + * @param hadoopConf The Hadoop configuration for Sequence File Writer. + * @param keyClass The class of key to write. + * @param valueClass The class of value to write. + */ + public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Class valueClass) { + this(hadoopConf, keyClass, valueClass, "None", SequenceFile.CompressionType.BLOCK); + } + + /** + * Creates a new SequenceFileWriterFactory using the given builder to assemble the + * SequenceFileWriter. + * + * @param hadoopConf The Hadoop configuration for Sequence File Writer. + * @param keyClass The class of key to write. + * @param valueClass The class of value to write. + * @param compressionCodecName The name of compression codec. + */ + public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Class valueClass, String compressionCodecName) { + this(hadoopConf, keyClass, valueClass, compressionCodecName, SequenceFile.CompressionType.BLOCK); + } + + /** + * Creates a new SequenceFileWriterFactory using the given builder to assemble the + * SequenceFileWriter. + * + * @param hadoopConf The Hadoop configuration for Sequence File Writer. + * @param keyClass The class of key to write. + * @param valueClass The class of value to write. + * @param compressionCodecName The name of compression codec. + * @param compressionType The type of compression level. + */ + public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Class valueClass, String compressionCodecName, SequenceFile.CompressionType compressionType) { + this.serdeHadoopConfig = new SerializableHadoopConfiguration(checkNotNull(hadoopConf)); + this.keyClass = checkNotNull(keyClass); + this.valueClass = checkNotNull(valueClass); + this.compressionCodecName = checkNotNull(compressionCodecName); + this.compressionType = checkNotNull(compressionType); + } + + @Override + public SequenceFileWriter create(FSDataOutputStream out) throws IOException { + org.apache.hadoop.fs.FSDataOutputStream stream = new org.apache.hadoop.fs.FSDataOutputStream(out, null); + CompressionCodec compressionCodec = getCompressionCodec(serdeHadoopConfig.get(), compressionCodecName); + SequenceFile.Writer writer = SequenceFile.createWriter( + serdeHadoopConfig.get(), + SequenceFile.Writer.stream(stream), + SequenceFile.Writer.keyClass(keyClass), + SequenceFile.Writer.valueClass(valueClass), + SequenceFile.Writer.compression(compressionType, compressionCodec)); + return new SequenceFileWriter<>(writer); + } + + private CompressionCodec getCompressionCodec(Configuration conf, String compressionCodecName) { + if (compressionCodecName.equals("None")) { + return null; + } + + CompressionCodecFactory codecFactory = new CompressionCodecFactory(checkNotNull(conf)); + CompressionCodec codec = codecFactory.getCodecByName(compressionCodecName); + if (codec == null) { + throw new RuntimeException("Codec " + compressionCodecName + " not found."); + } + return codec; + } + + /** + * Get Hadoop configuration based by the path. + * If the path is not Hadoop URI, it will be return default configuration. + * + * @param path The path to get configuration. + * @return Hadoop configuration. + * @throws IOException + */ + public static Configuration getHadoopConfigFromPath(Path path) throws IOException { + FileSystem fs = FileSystem.getUnguardedFileSystem(path.toUri()); + if (fs != null && fs instanceof HadoopFileSystem) { + return ((HadoopFileSystem) fs).getHadoopFileSystem().getConf(); + } else { + return new Configuration(); + } + } + + /** + * The wrapper class for serialization of {@link Configuration}. + */ + private class SerializableHadoopConfiguration implements Serializable { + private transient Configuration hadoopConfig; + + private SerializableHadoopConfiguration(Configuration hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + private Configuration get() { + return this.hadoopConfig; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + this.hadoopConfig.write(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + this.hadoopConfig = new Configuration(); + this.hadoopConfig.readFields(in); + } + } +} + diff --git a/flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.java b/flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.java new file mode 100644 index 0000000000000..b3d4b22ec9dce --- /dev/null +++ b/flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.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.formats.sequencefile; + +import org.apache.flink.api.common.functions.MapFunction; +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.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.test.util.FiniteTestSource; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Integration test case for writing bulk encoded files with the + * {@link StreamingFileSink} with SequenceFile. + */ +public class SequenceFileSinkITCase extends AbstractTestBase { + @Test + public void testWriteSequenceFile() throws Exception { + final File folder = TEMPORARY_FOLDER.newFolder(); + + final List> data = Arrays.asList( + new Tuple2<>(1L, "a"), + new Tuple2<>(2L, "b"), + new Tuple2<>(3L, "c") + ); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + + DataStream> stream = env.addSource( + new FiniteTestSource<>(data), TypeInformation.of(new TypeHint>() { + })); + + Path testPath = Path.fromLocalFile(folder); + + stream.map(new MapFunction, Tuple2>() { + @Override + public Tuple2 map(Tuple2 value) throws Exception { + return new Tuple2<>(new LongWritable(value.f0), new Text(value.f1)); + } + }).addSink( + StreamingFileSink.forBulkFormat( + testPath, + new SequenceFileWriterFactory<>(SequenceFileWriterFactory.getHadoopConfigFromPath(testPath), LongWritable.class, Text.class, "BZip2") + ).build()); + + env.execute(); + + validateResults(folder, data); + } + + private List> readSequenceFile(File file) throws IOException { + SequenceFile.Reader reader = new SequenceFile.Reader( + new Configuration(), SequenceFile.Reader.file(new org.apache.hadoop.fs.Path(file.toURI()))); + LongWritable key = new LongWritable(); + Text val = new Text(); + ArrayList> results = new ArrayList<>(); + while (reader.next(key, val)) { + results.add(new Tuple2<>(key.get(), val.toString())); + } + reader.close(); + return results; + } + + private void validateResults(File folder, List> expected) throws Exception { + File[] buckets = folder.listFiles(); + assertNotNull(buckets); + assertEquals(1, buckets.length); + + File[] partFiles = buckets[0].listFiles(); + assertNotNull(partFiles); + assertEquals(1, partFiles.length); + assertTrue(partFiles[0].length() > 0); + + List> results = readSequenceFile(partFiles[0]); + assertEquals(expected, results); + } +} + + diff --git a/flink-formats/flink-sequencefile/src/test/resources/log4j-test.properties b/flink-formats/flink-sequencefile/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..644b884d04d91 --- /dev/null +++ b/flink-formats/flink-sequencefile/src/test/resources/log4j-test.properties @@ -0,0 +1,23 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=OFF, testlogger +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target=System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/flink-formats/pom.xml b/flink-formats/pom.xml index c17ca58831e26..34ed0a0b229e9 100644 --- a/flink-formats/pom.xml +++ b/flink-formats/pom.xml @@ -40,6 +40,7 @@ under the License. flink-json flink-avro-confluent-registry flink-parquet + flink-sequencefile diff --git a/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java b/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java similarity index 97% rename from flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java rename to flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java index fa8fab240937e..169aa4ba7cbc9 100644 --- a/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java +++ b/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriter.java @@ -37,9 +37,10 @@ */ @PublicEvolving public class SequenceFileWriter implements BulkWriter> { + private final SequenceFile.Writer writer; - public SequenceFileWriter(SequenceFile.Writer writer) { + SequenceFileWriter(SequenceFile.Writer writer) { this.writer = checkNotNull(writer); } @@ -58,4 +59,3 @@ public void finish() throws IOException { writer.close(); } } - diff --git a/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java b/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java similarity index 71% rename from flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java rename to flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java index 90a87e77bcabe..d7b96f6700271 100644 --- a/flink-formats/flink-sequencefile/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java +++ b/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SequenceFileWriterFactory.java @@ -22,9 +22,6 @@ import org.apache.flink.api.common.serialization.BulkWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataOutputStream; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.SequenceFile; @@ -33,9 +30,6 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -47,9 +41,13 @@ */ @PublicEvolving public class SequenceFileWriterFactory implements BulkWriter.Factory> { + private static final long serialVersionUID = 1L; - private final SerializableHadoopConfiguration serdeHadoopConfig; + /** A constant specifying that no compression is requested. */ + public static final String NO_COMPRESSION = "NO_COMPRESSION"; + + private final SerializableHadoopConfiguration serializableHadoopConfig; private final Class keyClass; private final Class valueClass; private final String compressionCodecName; @@ -64,7 +62,7 @@ public class SequenceFileWriterFactory i * @param valueClass The class of value to write. */ public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Class valueClass) { - this(hadoopConf, keyClass, valueClass, "None", SequenceFile.CompressionType.BLOCK); + this(hadoopConf, keyClass, valueClass, NO_COMPRESSION, SequenceFile.CompressionType.BLOCK); } /** @@ -91,7 +89,7 @@ public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Cl * @param compressionType The type of compression level. */ public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Class valueClass, String compressionCodecName, SequenceFile.CompressionType compressionType) { - this.serdeHadoopConfig = new SerializableHadoopConfiguration(checkNotNull(hadoopConf)); + this.serializableHadoopConfig = new SerializableHadoopConfiguration(checkNotNull(hadoopConf)); this.keyClass = checkNotNull(keyClass); this.valueClass = checkNotNull(valueClass); this.compressionCodecName = checkNotNull(compressionCodecName); @@ -101,9 +99,9 @@ public SequenceFileWriterFactory(Configuration hadoopConf, Class keyClass, Cl @Override public SequenceFileWriter create(FSDataOutputStream out) throws IOException { org.apache.hadoop.fs.FSDataOutputStream stream = new org.apache.hadoop.fs.FSDataOutputStream(out, null); - CompressionCodec compressionCodec = getCompressionCodec(serdeHadoopConfig.get(), compressionCodecName); + CompressionCodec compressionCodec = getCompressionCodec(serializableHadoopConfig.get(), compressionCodecName); SequenceFile.Writer writer = SequenceFile.createWriter( - serdeHadoopConfig.get(), + serializableHadoopConfig.get(), SequenceFile.Writer.stream(stream), SequenceFile.Writer.keyClass(keyClass), SequenceFile.Writer.valueClass(valueClass), @@ -112,57 +110,19 @@ public SequenceFileWriter create(FSDataOutputStream out) throws IOExceptio } private CompressionCodec getCompressionCodec(Configuration conf, String compressionCodecName) { - if (compressionCodecName.equals("None")) { + checkNotNull(conf); + checkNotNull(compressionCodecName); + + if (compressionCodecName.equals(NO_COMPRESSION)) { return null; } - CompressionCodecFactory codecFactory = new CompressionCodecFactory(checkNotNull(conf)); + CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf); CompressionCodec codec = codecFactory.getCodecByName(compressionCodecName); if (codec == null) { throw new RuntimeException("Codec " + compressionCodecName + " not found."); } return codec; } - - /** - * Get Hadoop configuration based by the path. - * If the path is not Hadoop URI, it will be return default configuration. - * - * @param path The path to get configuration. - * @return Hadoop configuration. - * @throws IOException - */ - public static Configuration getHadoopConfigFromPath(Path path) throws IOException { - FileSystem fs = FileSystem.getUnguardedFileSystem(path.toUri()); - if (fs != null && fs instanceof HadoopFileSystem) { - return ((HadoopFileSystem) fs).getHadoopFileSystem().getConf(); - } else { - return new Configuration(); - } - } - - /** - * The wrapper class for serialization of {@link Configuration}. - */ - private class SerializableHadoopConfiguration implements Serializable { - private transient Configuration hadoopConfig; - - private SerializableHadoopConfiguration(Configuration hadoopConfig) { - this.hadoopConfig = hadoopConfig; - } - - private Configuration get() { - return this.hadoopConfig; - } - - private void writeObject(ObjectOutputStream out) throws IOException { - this.hadoopConfig.write(out); - } - - private void readObject(ObjectInputStream in) throws IOException { - this.hadoopConfig = new Configuration(); - this.hadoopConfig.readFields(in); - } - } } diff --git a/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SerializableHadoopConfiguration.java b/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SerializableHadoopConfiguration.java new file mode 100644 index 0000000000000..8e00e072b69d1 --- /dev/null +++ b/flink-formats/flink-sequence-file/src/main/java/org/apache/flink/formats/sequencefile/SerializableHadoopConfiguration.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.formats.sequencefile; + +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * Wrapper class for serialization of {@link Configuration}. + */ +class SerializableHadoopConfiguration implements Serializable { + + private static final long serialVersionUID = 1L; + + private transient Configuration hadoopConfig; + + SerializableHadoopConfiguration(Configuration hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + Configuration get() { + return this.hadoopConfig; + } + + // -------------------- + private void writeObject(ObjectOutputStream out) throws IOException { + this.hadoopConfig.write(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + final Configuration config = new Configuration(); + config.readFields(in); + + if (this.hadoopConfig == null) { + this.hadoopConfig = config; + } + } +} diff --git a/flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.java b/flink-formats/flink-sequence-file/src/test/java/org/apache/flink/formats/sequencefile/SequenceStreamingFileSinkITCase.java similarity index 78% rename from flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.java rename to flink-formats/flink-sequence-file/src/test/java/org/apache/flink/formats/sequencefile/SequenceStreamingFileSinkITCase.java index b3d4b22ec9dce..2b3f325579e92 100644 --- a/flink-formats/flink-sequencefile/src/test/java/org/apache/flink/formats/sequencefile/SequenceFileSinkITCase.java +++ b/flink-formats/flink-sequence-file/src/test/java/org/apache/flink/formats/sequencefile/SequenceStreamingFileSinkITCase.java @@ -26,8 +26,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; +import org.apache.flink.streaming.util.FiniteTestSource; import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.test.util.FiniteTestSource; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.LongWritable; @@ -49,26 +49,31 @@ * Integration test case for writing bulk encoded files with the * {@link StreamingFileSink} with SequenceFile. */ -public class SequenceFileSinkITCase extends AbstractTestBase { - @Test - public void testWriteSequenceFile() throws Exception { - final File folder = TEMPORARY_FOLDER.newFolder(); +public class SequenceStreamingFileSinkITCase extends AbstractTestBase { + + private final Configuration configuration = new Configuration(); - final List> data = Arrays.asList( + private final List> testData = Arrays.asList( new Tuple2<>(1L, "a"), new Tuple2<>(2L, "b"), new Tuple2<>(3L, "c") - ); + ); + + @Test + public void testWriteSequenceFile() throws Exception { + final File folder = TEMPORARY_FOLDER.newFolder(); + final Path testPath = Path.fromLocalFile(folder); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); env.enableCheckpointing(100); DataStream> stream = env.addSource( - new FiniteTestSource<>(data), TypeInformation.of(new TypeHint>() { - })); + new FiniteTestSource<>(testData), + TypeInformation.of(new TypeHint>() { - Path testPath = Path.fromLocalFile(folder); + }) + ); stream.map(new MapFunction, Tuple2>() { @Override @@ -78,17 +83,17 @@ public Tuple2 map(Tuple2 value) throws Excepti }).addSink( StreamingFileSink.forBulkFormat( testPath, - new SequenceFileWriterFactory<>(SequenceFileWriterFactory.getHadoopConfigFromPath(testPath), LongWritable.class, Text.class, "BZip2") + new SequenceFileWriterFactory<>(configuration, LongWritable.class, Text.class, "BZip2") ).build()); env.execute(); - validateResults(folder, data); + validateResults(folder, testData); } private List> readSequenceFile(File file) throws IOException { SequenceFile.Reader reader = new SequenceFile.Reader( - new Configuration(), SequenceFile.Reader.file(new org.apache.hadoop.fs.Path(file.toURI()))); + configuration, SequenceFile.Reader.file(new org.apache.hadoop.fs.Path(file.toURI()))); LongWritable key = new LongWritable(); Text val = new Text(); ArrayList> results = new ArrayList<>(); @@ -104,14 +109,15 @@ private void validateResults(File folder, List> expected) t assertNotNull(buckets); assertEquals(1, buckets.length); - File[] partFiles = buckets[0].listFiles(); + final File[] partFiles = buckets[0].listFiles(); assertNotNull(partFiles); - assertEquals(1, partFiles.length); - assertTrue(partFiles[0].length() > 0); + assertEquals(2, partFiles.length); + + for (File partFile : partFiles) { + assertTrue(partFile.length() > 0); - List> results = readSequenceFile(partFiles[0]); - assertEquals(expected, results); + final List> fileContent = readSequenceFile(partFile); + assertEquals(expected, fileContent); + } } } - - diff --git a/flink-formats/flink-sequence-file/src/test/java/org/apache/flink/formats/sequencefile/SerializableHadoopConfigurationTest.java b/flink-formats/flink-sequence-file/src/test/java/org/apache/flink/formats/sequencefile/SerializableHadoopConfigurationTest.java new file mode 100644 index 0000000000000..ea0fb955dc5a4 --- /dev/null +++ b/flink-formats/flink-sequence-file/src/test/java/org/apache/flink/formats/sequencefile/SerializableHadoopConfigurationTest.java @@ -0,0 +1,98 @@ +/* + * 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.formats.sequencefile; + +import org.apache.hadoop.conf.Configuration; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +/** + * Tests for the {@link SerializableHadoopConfiguration}. + */ +public class SerializableHadoopConfigurationTest { + + private static final String TEST_KEY = "test-key"; + + private static final String TEST_VALUE = "test-value"; + + private Configuration configuration; + + @Before + public void createConfigWithCustomProperty() { + this.configuration = new Configuration(); + configuration.set(TEST_KEY, TEST_VALUE); + } + + @Test + public void customPropertiesSurviveSerializationDeserialization() throws IOException, ClassNotFoundException { + final SerializableHadoopConfiguration serializableConfigUnderTest = new SerializableHadoopConfiguration(configuration); + final byte[] serializedConfigUnderTest = serializeAndGetBytes(serializableConfigUnderTest); + final SerializableHadoopConfiguration deserializableConfigUnderTest = deserializeAndGetConfiguration(serializedConfigUnderTest); + + Assert.assertThat(deserializableConfigUnderTest.get(), hasTheSamePropertiesAs(configuration)); + } + + // ---------------------------------------- Matchers ---------------------------------------- // + + private static TypeSafeMatcher hasTheSamePropertiesAs(final Configuration expectedConfig) { + return new TypeSafeMatcher() { + @Override + protected boolean matchesSafely(Configuration actualConfig) { + final String value = actualConfig.get(TEST_KEY); + return actualConfig != expectedConfig && value != null && expectedConfig.get(TEST_KEY).equals(value); + } + + @Override + public void describeTo(Description description) { + description.appendText("a Hadoop Configuration with property: key=") + .appendValue(TEST_KEY) + .appendText(" and value=") + .appendValue(TEST_VALUE); + } + }; + } + + // ---------------------------------------- Helper Methods ---------------------------------------- // + + private byte[] serializeAndGetBytes(SerializableHadoopConfiguration serializableConfigUnderTest) throws IOException { + try ( + ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(byteStream) + ) { + out.writeObject(serializableConfigUnderTest); + out.flush(); + return byteStream.toByteArray(); + } + } + + private SerializableHadoopConfiguration deserializeAndGetConfiguration(byte[] serializedConfig) throws IOException, ClassNotFoundException { + try (ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(serializedConfig))) { + return (SerializableHadoopConfiguration) in.readObject(); + } + } +} diff --git a/flink-formats/flink-sequencefile/src/test/resources/log4j-test.properties b/flink-formats/flink-sequence-file/src/test/resources/log4j-test.properties similarity index 100% rename from flink-formats/flink-sequencefile/src/test/resources/log4j-test.properties rename to flink-formats/flink-sequence-file/src/test/resources/log4j-test.properties diff --git a/flink-formats/pom.xml b/flink-formats/pom.xml index 34ed0a0b229e9..4b0d2f6133bfb 100644 --- a/flink-formats/pom.xml +++ b/flink-formats/pom.xml @@ -40,7 +40,7 @@ under the License. flink-json flink-avro-confluent-registry flink-parquet - flink-sequencefile + flink-sequence-file org.apache.maven.plugins From bb560c55c662675611a4921ff38da27cc435426d Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 19 Dec 2018 13:38:25 +0100 Subject: [PATCH 255/359] [FLINK-11194][hbase] Use type instead of classifier --- flink-connectors/flink-hbase/pom.xml | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml index bacc4ff7116be..c8c1007cb5b94 100644 --- a/flink-connectors/flink-hbase/pom.xml +++ b/flink-connectors/flink-hbase/pom.xml @@ -49,19 +49,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - - - - shade-flink - none - - - @@ -255,7 +242,7 @@ under the License. org.apache.hbase hbase-server ${hbase.version} - tests + test-jar test From ae832444320d546275ab1214df7255324d3b8fa9 Mon Sep 17 00:00:00 2001 From: EronWright Date: Sun, 30 Dec 2018 19:59:47 -0800 Subject: [PATCH 256/359] [FLINK-11234] [table] Fix ExternalTableCatalogBuilder unable to build a batch-only table - fix the logic in supportsBatch to properly declare a batch-only table - adjust CommonTestData to provide batch-only or streaming-only tables This closes #7386. --- .../flink/table/catalog/ExternalCatalogTable.scala | 4 ++-- .../flink/table/runtime/utils/CommonTestData.scala | 12 +++++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala index 45414ee3ff6e7..ce57070ac98d7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala @@ -270,8 +270,8 @@ class ExternalCatalogTableBuilder(private val connectorDescriptor: ConnectorDesc * Explicitly declares this external table for supporting only batch environments. */ def supportsBatch(): ExternalCatalogTableBuilder = { - isBatch = false - isStreaming = true + isBatch = true + isStreaming = false this } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala index 64fcc8ac7c480..1209595837bf3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/CommonTestData.scala @@ -85,7 +85,9 @@ object CommonTestData { .withSchema(schemaDesc1) if (isStreaming) { - externalTableBuilder1.inAppendMode() + externalTableBuilder1.supportsStreaming().inAppendMode() + } else { + externalTableBuilder1.supportsBatch() } val csvRecord2 = Seq( @@ -126,7 +128,9 @@ object CommonTestData { .withSchema(schemaDesc2) if (isStreaming) { - externalTableBuilder2.inAppendMode() + externalTableBuilder2.supportsStreaming().inAppendMode() + } else { + externalTableBuilder2.supportsBatch() } val tempFilePath3 = writeToTempFile("", "csv-test3", "tmp") @@ -145,7 +149,9 @@ object CommonTestData { .withSchema(schemaDesc3) if (isStreaming) { - externalTableBuilder3.inAppendMode() + externalTableBuilder3.supportsStreaming().inAppendMode() + } else { + externalTableBuilder3.supportsBatch() } val catalog = new InMemoryExternalCatalog("test") From b6ce4e47b0e72befba72f2e1faa2c901876a2182 Mon Sep 17 00:00:00 2001 From: EronWright Date: Mon, 31 Dec 2018 09:02:17 -0800 Subject: [PATCH 257/359] [hotfix] [table] Fix typos in Table javadoc. This closes #7388. --- .../apache/flink/table/descriptors/ConnectorDescriptor.java | 2 +- .../table/descriptors/ConnectorDescriptorValidator.java | 5 +++++ .../apache/flink/table/factories/BatchTableSinkFactory.scala | 2 +- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptor.java b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptor.java index a1d3e783cdc1c..ed62b46b36b50 100644 --- a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptor.java +++ b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptor.java @@ -68,7 +68,7 @@ protected final boolean isFormatNeeded() { /** * Converts this descriptor into a set of connector properties. Usually prefixed with - * {@link FormatDescriptorValidator#FORMAT}. + * {@link ConnectorDescriptorValidator#CONNECTOR}. */ protected abstract Map toConnectorProperties(); } diff --git a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.java b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.java index 88cf34905f452..cf9860ae69e2a 100644 --- a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.java +++ b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.java @@ -26,6 +26,11 @@ @Internal public abstract class ConnectorDescriptorValidator implements DescriptorValidator { + /** + * Prefix for connector-related properties. + */ + public static final String CONNECTOR = "connector"; + /** * Key for describing the type of the connector. Usually used for factory discovery. */ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala index 6fd1f7afb16b5..0b685239e05aa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala @@ -23,7 +23,7 @@ import java.util import org.apache.flink.table.sinks.BatchTableSink /** - * A factory to create configured table sink instances in a streaming environment based on + * A factory to create configured table sink instances in a batch environment based on * string-based properties. See also [[TableFactory]] for more information. * * @tparam T type of records that the factory consumes From f9c587475abb0c6aeb646b068602a1d7cb2e5cf3 Mon Sep 17 00:00:00 2001 From: wenhuitang Date: Sat, 29 Dec 2018 18:50:31 +0800 Subject: [PATCH 258/359] [FLINK-11119][docs] Correct Scala example for Table Function in User-defined Functions docs. This closes #7379. --- docs/dev/table/udfs.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/udfs.md b/docs/dev/table/udfs.md index 20bf49d5999b1..47c44da390ba5 100644 --- a/docs/dev/table/udfs.md +++ b/docs/dev/table/udfs.md @@ -190,7 +190,7 @@ tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(s class Split(separator: String) extends TableFunction[(String, Int)] { def eval(str: String): Unit = { // use collect(...) to emit a row. - str.split(separator).foreach(x -> collect((x, x.length)) + str.split(separator).foreach(x => collect((x, x.length))) } } @@ -210,7 +210,7 @@ tableEnv.registerFunction("split", new Split("#")) // CROSS JOIN a table function (equivalent to "join" in Table API) tableEnv.sqlQuery("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)") // LEFT JOIN a table function (equivalent to "leftOuterJoin" in Table API) -tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN TABLE(split(a)) as T(word, length) ON TRUE") +tableEnv.sqlQuery("SELECT a, word, length FROM MyTable LEFT JOIN LATERAL TABLE(split(a)) as T(word, length) ON TRUE") {% endhighlight %} **IMPORTANT:** Do not implement TableFunction as a Scala object. Scala object is a singleton and will cause concurrency issues.
    From 8ee0465761d58eb7fd96846c09614410d644ed65 Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Tue, 25 Dec 2018 12:03:39 +0800 Subject: [PATCH 259/359] [FLINK-11227] [table] Fix bound checking errors in DescriptorProperties This closes #7373. --- .../descriptors/DescriptorProperties.java | 4 +- .../DescriptorPropertiesTest.scala | 45 +++++++++++++++++-- 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java index c47698147284e..76289128b5bca 100644 --- a/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java +++ b/flink-libraries/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java @@ -938,7 +938,7 @@ public void validateFixedIndexedProperties(String key, boolean allowEmpty, Map> subKey : subKeyValidation.entrySet()) { final String fullKey = key + '.' + i + '.' + subKey.getKey(); if (properties.containsKey(fullKey)) { @@ -1134,7 +1134,7 @@ public void validateArray(String key, Consumer elementValidation, int mi } // validate array elements - for (int i = 0; i < maxIndex; i++) { + for (int i = 0; i <= maxIndex; i++) { final String fullKey = key + '.' + i; if (properties.containsKey(fullKey)) { // run validation logic diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorPropertiesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorPropertiesTest.scala index b2a8ec9f567d8..fe7c75df6cf78 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorPropertiesTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorPropertiesTest.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.descriptors import java.util import java.util.Collections +import java.util.function.Consumer import org.apache.flink.table.api.ValidationException import org.apache.flink.table.util.JavaScalaConversionUtil.toJava @@ -32,6 +33,9 @@ import org.junit.Test class DescriptorPropertiesTest { private val ARRAY_KEY = "my-array" + private val FIXED_INDEXED_PROPERTY_KEY = "my-fixed-indexed-property" + private val PROPERTY_1_KEY = "property-1" + private val PROPERTY_2_KEY = "property-2" @Test def testEquals(): Unit = { @@ -97,8 +101,8 @@ class DescriptorPropertiesTest { def testArrayInvalidValues(): Unit = { val properties = new DescriptorProperties() properties.putString(s"$ARRAY_KEY.0", "12") - properties.putString(s"$ARRAY_KEY.1", "INVALID") - properties.putString(s"$ARRAY_KEY.2", "66") + properties.putString(s"$ARRAY_KEY.1", "66") + properties.putString(s"$ARRAY_KEY.2", "INVALID") testArrayValidation(properties, 1, Integer.MAX_VALUE) } @@ -118,6 +122,19 @@ class DescriptorPropertiesTest { testArrayValidation(properties, 1, Integer.MAX_VALUE) } + @Test(expected = classOf[ValidationException]) + def testInvalidFixedIndexedProperties(): Unit = { + val property = new DescriptorProperties() + val list = new util.ArrayList[util.List[String]]() + list.add(util.Arrays.asList("1", "string")) + list.add(util.Arrays.asList("INVALID", "string")) + property.putIndexedFixedProperties( + FIXED_INDEXED_PROPERTY_KEY, + util.Arrays.asList(PROPERTY_1_KEY, PROPERTY_2_KEY), + list) + testFixedIndexedPropertiesValidation(property) + } + @Test def testRemoveKeys(): Unit = { val properties = new DescriptorProperties() @@ -155,7 +172,7 @@ class DescriptorPropertiesTest { minLength: Int, maxLength: Int) : Unit = { - val validator: (String) => Unit = (key: String) => { + val validator: String => Unit = (key: String) => { properties.validateInt(key, false) } @@ -165,4 +182,26 @@ class DescriptorPropertiesTest { minLength, maxLength) } + + private def testFixedIndexedPropertiesValidation(properties: DescriptorProperties): Unit = { + + val validatorMap = new util.HashMap[String, Consumer[String]]() + + // PROPERTY_1 should be Int + val validator1: String => Unit = (key: String) => { + properties.validateInt(key, false) + } + validatorMap.put(PROPERTY_1_KEY, toJava(validator1)) + // PROPERTY_2 should be String + val validator2: String => Unit = (key: String) => { + properties.validateString(key, false) + } + validatorMap.put(PROPERTY_2_KEY, toJava(validator2)) + + properties.validateFixedIndexedProperties( + FIXED_INDEXED_PROPERTY_KEY, + false, + validatorMap + ) + } } From e2daaaaba59e79cbef6bbf80309beb097310c2fb Mon Sep 17 00:00:00 2001 From: wenhuitang Date: Tue, 25 Dec 2018 20:16:27 +0800 Subject: [PATCH 260/359] [FLINK-11173][table] Fix the exception message of proctime attribute validation in TableSourceUtil#validateTableSource This closes #7374. --- .../org/apache/flink/table/sources/TableSourceUtil.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala index 15d68900d219d..998fc1ff34083 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala @@ -114,12 +114,12 @@ object TableSourceUtil { val rowtimeIdx = schema.getFieldNames.indexOf(rowtimeAttribute) // ensure that field exists if (rowtimeIdx < 0) { - throw new ValidationException(s"Found a RowtimeAttributeDescriptor for field " + + throw new ValidationException(s"Found a rowtime attribute for field " + s"'$rowtimeAttribute' but field '$rowtimeAttribute' does not exist in table.") } // ensure that field is of type TIMESTAMP if (schema.getFieldTypes()(rowtimeIdx) != Types.SQL_TIMESTAMP) { - throw new ValidationException(s"Found a RowtimeAttributeDescriptor for field " + + throw new ValidationException(s"Found a rowtime attribute for field " + s"'$rowtimeAttribute' but field '$rowtimeAttribute' is not of type TIMESTAMP.") } // look up extractor input fields in return type @@ -138,12 +138,12 @@ object TableSourceUtil { val proctimeIdx = schema.getFieldNames.indexOf(proctimeAttribute) // ensure that field exists if (proctimeIdx < 0) { - throw new ValidationException(s"Found a RowtimeAttributeDescriptor for field " + + throw new ValidationException(s"Found a processing time attribute for field " + s"'$proctimeAttribute' but field '$proctimeAttribute' does not exist in table.") } // ensure that field is of type TIMESTAMP if (schema.getFieldTypes()(proctimeIdx) != Types.SQL_TIMESTAMP) { - throw new ValidationException(s"Found a RowtimeAttributeDescriptor for field " + + throw new ValidationException(s"Found a processing time attribute for field " + s"'$proctimeAttribute' but field '$proctimeAttribute' is not of type TIMESTAMP.") } case _ => // nothing to validate From 2576076f36e75fa81896a7cc275315bd8cd848da Mon Sep 17 00:00:00 2001 From: Shuyi Chen Date: Sat, 10 Nov 2018 00:42:49 -0800 Subject: [PATCH 261/359] [FLINK-10848][YARN] properly remove YARN ContainerRequest upon container allocation success This closes #7078 --- .../test/java/org/apache/flink/yarn/YarnTestBase.java | 2 ++ .../org/apache/flink/yarn/YarnFlinkResourceManager.java | 2 ++ .../java/org/apache/flink/yarn/YarnResourceManager.java | 3 ++- .../apache/flink/yarn/YarnFlinkResourceManagerTest.java | 9 +++++++++ .../org/apache/flink/yarn/YarnResourceManagerTest.java | 4 ++++ 5 files changed, 19 insertions(+), 1 deletion(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 3763f6592afee..f1e6a3a767cd9 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -165,6 +165,8 @@ public abstract class YarnTestBase extends TestLogger { YARN_CONFIGURATION.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600); YARN_CONFIGURATION.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false); YARN_CONFIGURATION.setInt(YarnConfiguration.NM_VCORES, 666); // memory is overwritten in the MiniYARNCluster. + YARN_CONFIGURATION.set("yarn.scheduler.capacity.resource-calculator", + "org.apache.hadoop.yarn.util.resource.DominantResourceCalculator"); // so we have to change the number of cores for testing. YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 20000); // 20 seconds expiry (to ensure we properly heartbeat with YARN). } 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 8e686bbbe34c1..3327505e32df5 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 @@ -438,6 +438,8 @@ private void containersAllocated(List containers) { numPendingContainerRequests = Math.max(0, numPendingContainerRequests - 1); LOG.info("Received new container: {} - Remaining pending container requests: {}", container.getId(), numPendingContainerRequests); + resourceManagerClient.removeContainerRequest(new AMRMClient.ContainerRequest( + container.getResource(), null, null, container.getPriority())); // decide whether to return the container, or whether to start a TaskManager if (numRegistered + containersInLaunch.size() < numRequired) { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 6ff5cd6648711..6669f16fa405d 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -361,7 +361,8 @@ public void onContainersAllocated(List containers) { "Received new container: {} - Remaining pending container requests: {}", container.getId(), numPendingContainerRequests); - + resourceManagerClient.removeContainerRequest(new AMRMClient.ContainerRequest( + container.getResource(), null, null, container.getPriority())); if (numPendingContainerRequests > 0) { numPendingContainerRequests--; diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java index 10b2ce97d6fe4..d665df6bc7ce0 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java @@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; @@ -69,8 +71,11 @@ import scala.concurrent.duration.FiniteDuration; import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** @@ -125,6 +130,8 @@ public void testYarnFlinkResourceManagerJobManagerLostLeadership() throws Except 1), i)); when(mockContainer.getNodeId()).thenReturn(NodeId.newInstance("container", 1234)); + when(mockContainer.getResource()).thenReturn(Resource.newInstance(200, 1)); + when(mockContainer.getPriority()).thenReturn(Priority.UNDEFINED); containerList.add(mockContainer); } @@ -233,6 +240,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { int numberOfRegisteredResources = (Integer) Await.result(numberOfRegisteredResourcesFuture, deadline.timeLeft()); + verify(resourceManagerClient, times(numInitialTaskManagers)).removeContainerRequest( + any(AMRMClient.ContainerRequest.class)); assertEquals(numInitialTaskManagers, numberOfRegisteredResources); } finally { if (resourceManager != null) { diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java index d41d42d7a05f9..ee325dad0f7b1 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java @@ -401,6 +401,8 @@ public void testStopWorker() throws Exception { resourceManager.onContainersAllocated(ImmutableList.of(testingContainer)); verify(mockResourceManagerClient).addContainerRequest(any(AMRMClient.ContainerRequest.class)); + verify(mockResourceManagerClient).removeContainerRequest( + any(AMRMClient.ContainerRequest.class)); verify(mockNMClient).startContainer(eq(testingContainer), any(ContainerLaunchContext.class)); // Remote task executor registers with YarnResourceManager. @@ -496,6 +498,8 @@ public void testOnContainerCompleted() throws Exception { resourceManager.onContainersAllocated(ImmutableList.of(testingContainer)); verify(mockResourceManagerClient).addContainerRequest(any(AMRMClient.ContainerRequest.class)); + verify(mockResourceManagerClient).removeContainerRequest( + any(AMRMClient.ContainerRequest.class)); verify(mockNMClient).startContainer(eq(testingContainer), any(ContainerLaunchContext.class)); // Callback from YARN when container is Completed, pending request can not be fulfilled by pending From 6528a3a5e651d9e98a028983f749e78645a078ca Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Tue, 8 Jan 2019 12:45:26 +0800 Subject: [PATCH 262/359] [FLINK-11279] [table] Fix week interval parsing in ExpressionParser This closes #7426. --- .../apache/flink/table/expressions/ExpressionParser.scala | 2 +- .../apache/flink/table/expressions/ScalarFunctionsTest.scala | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala index 7fd9309b5dbb1..0465e37f51876 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala @@ -303,7 +303,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { case expr ~ _ ~ (MONTHS.key | MONTH.key) => toMonthInterval(expr, 1) - case expr ~ _ ~ (WEEKS.key | WEEKS.key) => toMilliInterval(expr, 7 * MILLIS_PER_DAY) + case expr ~ _ ~ (WEEKS.key | WEEK.key) => toMilliInterval(expr, 7 * MILLIS_PER_DAY) case expr ~ _ ~ (DAYS.key | DAY.key) => toMilliInterval(expr, MILLIS_PER_DAY) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala index 31c9b6c3d0167..23bd2ae337998 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala @@ -2764,6 +2764,11 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { "'2016-03-31'.toDate - 1.month", "timestampadd(MONTH, -1, date '2016-03-31')", "2016-02-29") + + testAllApis("2016-03-31".toDate - 1.week, + "'2016-03-31'.toDate - 1.week", + "timestampadd(WEEK, -1, date '2016-03-31')", + "2016-03-24") } // ---------------------------------------------------------------------------------------------- From b195472c0e513bc8afc36c5761edf2680e7a6f5b Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 18 Dec 2018 15:37:42 +0100 Subject: [PATCH 263/359] [FLINK-11191] [table] Check for ambiguous columns in MATCH_RECOGNIZE Added a validation that checks if no ambiguous columns are defined in MATCH_RECOGNIZE clause. Without the check there is a cryptic message thrown from code generation stack. This closes #7328. --- .../datastream/DataStreamMatchRule.scala | 61 ++++++++++++++++++- .../match/MatchOperatorValidationTest.scala | 29 +++++++++ 2 files changed, 88 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala index 5b0aa65362f5a..5c0241f55e705 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala @@ -18,15 +18,23 @@ package org.apache.flink.table.plan.rules.datastream -import org.apache.calcite.plan.{RelOptRule, RelTraitSet} +import java.util.{List => JList} + +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.convert.ConverterRule -import org.apache.flink.table.api.TableException +import org.apache.calcite.rex.{RexInputRef, RexNode} +import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.plan.logical.MatchRecognize import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.DataStreamMatch import org.apache.flink.table.plan.nodes.logical.FlinkLogicalMatch import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.plan.util.RexDefaultVisitor + +import scala.collection.JavaConverters._ +import scala.collection.mutable class DataStreamMatchRule extends ConverterRule( @@ -35,6 +43,14 @@ class DataStreamMatchRule FlinkConventions.DATASTREAM, "DataStreamMatchRule") { + override def matches(call: RelOptRuleCall): Boolean = { + val logicalMatch: FlinkLogicalMatch = call.rel(0).asInstanceOf[FlinkLogicalMatch] + + // This check might be obsolete once CALCITE-2747 is resolved + validateAmbiguousColumns(logicalMatch) + true + } + override def convert(rel: RelNode): RelNode = { val logicalMatch: FlinkLogicalMatch = rel.asInstanceOf[FlinkLogicalMatch] val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) @@ -71,6 +87,47 @@ class DataStreamMatchRule new RowSchema(logicalMatch.getRowType), new RowSchema(logicalMatch.getInput.getRowType)) } + + private def validateAmbiguousColumns(logicalMatch: FlinkLogicalMatch): Unit = { + if (logicalMatch.isAllRows) { + throw new TableException("All rows per match mode is not supported yet.") + } else { + val refNameFinder = new RefNameFinder(logicalMatch.getInput.getRowType) + validateAmbiguousColumnsOnRowPerMatch( + logicalMatch.getPartitionKeys, + logicalMatch.getMeasures.keySet().asScala, + logicalMatch.getRowType, + refNameFinder) + } + } + + private def validateAmbiguousColumnsOnRowPerMatch( + partitionKeys: JList[RexNode], + measuresNames: mutable.Set[String], + expectedSchema: RelDataType, + refNameFinder: RefNameFinder) + : Unit = { + val actualSize = partitionKeys.size() + measuresNames.size + val expectedSize = expectedSchema.getFieldCount + if (actualSize != expectedSize) { + //try to find ambiguous column + + val ambiguousColumns = partitionKeys.asScala.map(_.accept(refNameFinder)) + .filter(measuresNames.contains).mkString("{", ", ", "}") + + throw new ValidationException(s"Columns ambiguously defined: $ambiguousColumns") + } + } + + private class RefNameFinder(inputSchema: RelDataType) extends RexDefaultVisitor[String] { + + override def visitInputRef(inputRef: RexInputRef): String = { + inputSchema.getFieldList.get(inputRef.getIndex).getName + } + + override def visitNode(rexNode: RexNode): String = + throw new TableException(s"PARTITION BY clause accepts only input reference. Found $rexNode") + } } object DataStreamMatchRule { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/MatchOperatorValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/MatchOperatorValidationTest.scala index 3917bdf8af934..4fa8b175a0e72 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/MatchOperatorValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/match/MatchOperatorValidationTest.scala @@ -179,6 +179,35 @@ class MatchOperatorValidationTest extends TableTestBase { streamUtils.tableEnv.sqlQuery(sqlQuery).toAppendStream[Row] } + @Test + def testValidatingAmbiguousColumns(): Unit = { + thrown.expectMessage("Columns ambiguously defined: {symbol, price}") + thrown.expect(classOf[ValidationException]) + + val sqlQuery = + s""" + |SELECT * + |FROM Ticker + |MATCH_RECOGNIZE ( + | PARTITION BY symbol, price + | ORDER BY proctime + | MEASURES + | A.symbol AS symbol, + | A.price AS price + | PATTERN (A) + | DEFINE + | A AS symbol = 'a' + |) AS T + |""".stripMargin + + streamUtils.tableEnv.sqlQuery(sqlQuery).toAppendStream[Row] + } + + // *************************************************************************************** + // * Those validations are temporary. We should remove those tests once we support those * + // * features. * + // *************************************************************************************** + @Test def testPatternsProducingEmptyMatchesAreNotSupported(): Unit = { thrown.expectMessage("Patterns that can produce empty matches are not supported. " + From fb0f811e4b4d395e182a124cecac7cdf845a8023 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 8 Jan 2019 11:24:38 +0100 Subject: [PATCH 264/359] Revert "[FLINK-10848][YARN] properly remove YARN ContainerRequest upon container allocation success" This reverts commit 2576076f36e75fa81896a7cc275315bd8cd848da. --- .../test/java/org/apache/flink/yarn/YarnTestBase.java | 2 -- .../org/apache/flink/yarn/YarnFlinkResourceManager.java | 2 -- .../java/org/apache/flink/yarn/YarnResourceManager.java | 3 +-- .../apache/flink/yarn/YarnFlinkResourceManagerTest.java | 9 --------- .../org/apache/flink/yarn/YarnResourceManagerTest.java | 4 ---- 5 files changed, 1 insertion(+), 19 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index f1e6a3a767cd9..3763f6592afee 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -165,8 +165,6 @@ public abstract class YarnTestBase extends TestLogger { YARN_CONFIGURATION.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 3600); YARN_CONFIGURATION.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false); YARN_CONFIGURATION.setInt(YarnConfiguration.NM_VCORES, 666); // memory is overwritten in the MiniYARNCluster. - YARN_CONFIGURATION.set("yarn.scheduler.capacity.resource-calculator", - "org.apache.hadoop.yarn.util.resource.DominantResourceCalculator"); // so we have to change the number of cores for testing. YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 20000); // 20 seconds expiry (to ensure we properly heartbeat with YARN). } 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 3327505e32df5..8e686bbbe34c1 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 @@ -438,8 +438,6 @@ private void containersAllocated(List containers) { numPendingContainerRequests = Math.max(0, numPendingContainerRequests - 1); LOG.info("Received new container: {} - Remaining pending container requests: {}", container.getId(), numPendingContainerRequests); - resourceManagerClient.removeContainerRequest(new AMRMClient.ContainerRequest( - container.getResource(), null, null, container.getPriority())); // decide whether to return the container, or whether to start a TaskManager if (numRegistered + containersInLaunch.size() < numRequired) { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 6669f16fa405d..6ff5cd6648711 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -361,8 +361,7 @@ public void onContainersAllocated(List containers) { "Received new container: {} - Remaining pending container requests: {}", container.getId(), numPendingContainerRequests); - resourceManagerClient.removeContainerRequest(new AMRMClient.ContainerRequest( - container.getResource(), null, null, container.getPriority())); + if (numPendingContainerRequests > 0) { numPendingContainerRequests--; diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java index d665df6bc7ce0..10b2ce97d6fe4 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java @@ -43,8 +43,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.Priority; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; @@ -71,11 +69,8 @@ import scala.concurrent.duration.FiniteDuration; import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** @@ -130,8 +125,6 @@ public void testYarnFlinkResourceManagerJobManagerLostLeadership() throws Except 1), i)); when(mockContainer.getNodeId()).thenReturn(NodeId.newInstance("container", 1234)); - when(mockContainer.getResource()).thenReturn(Resource.newInstance(200, 1)); - when(mockContainer.getPriority()).thenReturn(Priority.UNDEFINED); containerList.add(mockContainer); } @@ -240,8 +233,6 @@ public Object answer(InvocationOnMock invocation) throws Throwable { int numberOfRegisteredResources = (Integer) Await.result(numberOfRegisteredResourcesFuture, deadline.timeLeft()); - verify(resourceManagerClient, times(numInitialTaskManagers)).removeContainerRequest( - any(AMRMClient.ContainerRequest.class)); assertEquals(numInitialTaskManagers, numberOfRegisteredResources); } finally { if (resourceManager != null) { diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java index ee325dad0f7b1..d41d42d7a05f9 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java @@ -401,8 +401,6 @@ public void testStopWorker() throws Exception { resourceManager.onContainersAllocated(ImmutableList.of(testingContainer)); verify(mockResourceManagerClient).addContainerRequest(any(AMRMClient.ContainerRequest.class)); - verify(mockResourceManagerClient).removeContainerRequest( - any(AMRMClient.ContainerRequest.class)); verify(mockNMClient).startContainer(eq(testingContainer), any(ContainerLaunchContext.class)); // Remote task executor registers with YarnResourceManager. @@ -498,8 +496,6 @@ public void testOnContainerCompleted() throws Exception { resourceManager.onContainersAllocated(ImmutableList.of(testingContainer)); verify(mockResourceManagerClient).addContainerRequest(any(AMRMClient.ContainerRequest.class)); - verify(mockResourceManagerClient).removeContainerRequest( - any(AMRMClient.ContainerRequest.class)); verify(mockNMClient).startContainer(eq(testingContainer), any(ContainerLaunchContext.class)); // Callback from YARN when container is Completed, pending request can not be fulfilled by pending From 3eaee6f46102ea6acd9c329cb870965442afafc4 Mon Sep 17 00:00:00 2001 From: Bo WANG <42056379+eaglewatcherwb@users.noreply.github.com> Date: Wed, 9 Jan 2019 16:05:13 +0800 Subject: [PATCH 265/359] [FLINK-11232][rest] Fix subtask start-time field name --- .../runtime/rest/messages/JobVertexDetailsInfo.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java index df007f45c0759..3ad1e411d29c8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobVertexDetailsInfo.java @@ -110,7 +110,8 @@ public static final class VertexTaskDetail { public static final String FIELD_NAME_STATUS = "status"; public static final String FIELD_NAME_ATTEMPT = "attempt"; public static final String FIELD_NAME_HOST = "host"; - public static final String FIELD_NAME_START_TIME = "start_time"; + public static final String FIELD_NAME_START_TIME = "start-time"; + public static final String FIELD_NAME_COMPATIBLE_START_TIME = "start_time"; public static final String FIELD_NAME_END_TIME = "end-time"; public static final String FIELD_NAME_DURATION = "duration"; public static final String FIELD_NAME_METRICS = "metrics"; @@ -130,6 +131,9 @@ public static final class VertexTaskDetail { @JsonProperty(FIELD_NAME_START_TIME) private final long startTime; + @JsonProperty(FIELD_NAME_COMPATIBLE_START_TIME) + private final long startTimeCompatible; + @JsonProperty(FIELD_NAME_END_TIME) private final long endTime; @@ -154,6 +158,7 @@ public VertexTaskDetail( this.attempt = attempt; this.host = checkNotNull(host); this.startTime = startTime; + this.startTimeCompatible = startTime; this.endTime = endTime; this.duration = duration; this.metrics = checkNotNull(metrics); @@ -175,6 +180,7 @@ public boolean equals(Object o) { attempt == that.attempt && Objects.equals(host, that.host) && startTime == that.startTime && + startTimeCompatible == that.startTimeCompatible && endTime == that.endTime && duration == that.duration && Objects.equals(metrics, that.metrics); @@ -182,7 +188,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(subtask, status, attempt, host, startTime, endTime, duration, metrics); + return Objects.hash(subtask, status, attempt, host, startTime, startTimeCompatible, endTime, duration, metrics); } } } From fdf7cb794604e72d0c7fe7cbf7f6fb18aff0e5ed Mon Sep 17 00:00:00 2001 From: Tony Wei Date: Wed, 9 Jan 2019 16:12:08 +0800 Subject: [PATCH 266/359] [FLINK-11251][metrics] Exclude variable values from logical scope of generic groups --- .../metrics/groups/AbstractMetricGroup.java | 2 +- .../groups/GenericValueMetricGroup.java | 2 +- .../metrics/groups/MetricGroupTest.java | 33 +++++++++++++++++++ 3 files changed, 35 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java index fb321303222c7..42e1cbbc0b71a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java @@ -172,7 +172,7 @@ String getLogicalScope(CharacterFilter filter, char delimiter, int reporterIndex } } - private String createLogicalScope(CharacterFilter filter, char delimiter) { + protected String createLogicalScope(CharacterFilter filter, char delimiter) { final String groupName = getGroupName(filter); return parent == null ? groupName diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/GenericValueMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/GenericValueMetricGroup.java index ef8e6e8bee8a2..41e9a91d38678 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/GenericValueMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/GenericValueMetricGroup.java @@ -51,7 +51,7 @@ protected void putVariables(Map variables) { } @Override - public String getLogicalScope(CharacterFilter filter, char delimiter) { + protected String createLogicalScope(CharacterFilter filter, char delimiter) { return parent.getLogicalScope(filter, delimiter); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java index 71ae7f18ecb7e..41d3fc9cb2bef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/MetricGroupTest.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.metrics.groups; import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.CharacterFilter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; @@ -31,6 +33,7 @@ import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormat; import org.apache.flink.runtime.metrics.util.DummyCharacterFilter; +import org.apache.flink.runtime.metrics.util.TestReporter; import org.apache.flink.util.AbstractID; import org.apache.flink.util.TestLogger; @@ -38,10 +41,13 @@ import org.junit.Before; import org.junit.Test; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -216,6 +222,33 @@ public void testNameCollisionAfterKeyValueGroup() { assertFalse("Value is present in logical scope.", logicalScope.contains(value)); } + /** + * Verifies that calling {@link AbstractMetricGroup#getLogicalScope(CharacterFilter, char, int)} on {@link GenericValueMetricGroup} + * should ignore value as well. + */ + @Test + public void testLogicalScopeShouldIgnoreValueGroupName() throws Exception { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config)); + try { + GenericMetricGroup root = new GenericMetricGroup(registry, new DummyAbstractMetricGroup(registry), "root"); + + String key = "key"; + String value = "value"; + + MetricGroup group = root.addGroup(key, value); + + String logicalScope = ((AbstractMetricGroup) group) + .getLogicalScope(new DummyCharacterFilter(), registry.getDelimiter(), 0); + assertThat("Key is missing from logical scope.", logicalScope, containsString(key)); + assertThat("Value is present in logical scope.", logicalScope, not(containsString(value))); + } finally { + registry.shutdown().get(); + } + } + @Test public void closedGroupDoesNotRegisterMetrics() { GenericMetricGroup group = new GenericMetricGroup( From 8e94ec612e2c7cdf70b152ba0f4893ed67bec329 Mon Sep 17 00:00:00 2001 From: zy <602128569@qq.com> Date: Wed, 9 Jan 2019 17:16:19 +0800 Subject: [PATCH 267/359] [hotfix][docs] Fix javadocs of RandomFibonacciSource --- .../flink/streaming/examples/iteration/IterateExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java index d123615c7fca2..13db58ae187b1 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java @@ -115,7 +115,7 @@ public static void main(String[] args) throws Exception { // ************************************************************************* /** - * Generate BOUND number of random integer pairs from the range from 0 to BOUND/2. + * Generate BOUND number of random integer pairs from the range from 1 to BOUND/2. */ private static class RandomFibonacciSource implements SourceFunction> { private static final long serialVersionUID = 1L; From 3f190b30eb94959992108d91aeca686e3873de03 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Tue, 8 Jan 2019 14:43:19 +0100 Subject: [PATCH 268/359] [FLINK-11140][fs-connector] Fix empty child path check in Buckets. --- .../functions/sink/filesystem/Buckets.java | 9 ++- .../filesystem/BucketAssignerITCases.java | 66 +++++++++++++++++++ .../sink/filesystem/BucketsTest.java | 10 +-- 3 files changed, 78 insertions(+), 7 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java index d08bc2ac0c382..7b35e50283065 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java @@ -254,7 +254,7 @@ private void snapshotActiveBuckets( } } - void onElement(final IN value, final SinkFunction.Context context) throws Exception { + Bucket onElement(final IN value, final SinkFunction.Context context) throws Exception { final long currentProcessingTime = context.currentProcessingTime(); // setting the values in the bucketer context @@ -272,6 +272,7 @@ void onElement(final IN value, final SinkFunction.Context context) throws Except // another part file for the bucket, if we start from 0 we may overwrite previous parts. this.maxPartCounter = Math.max(maxPartCounter, bucket.getPartCounter()); + return bucket; } private Bucket getOrCreateBucketForBucketId(final BucketID bucketId) throws IOException { @@ -304,7 +305,11 @@ void close() { } private Path assembleBucketPath(BucketID bucketId) { - return new Path(basePath, bucketId.toString()); + final String child = bucketId.toString(); + if ("".equals(child)) { + return basePath; + } + return new Path(basePath, child); } /** diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java new file mode 100644 index 0000000000000..5ae57ce43bc21 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.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.streaming.api.functions.sink.filesystem; + +import org.apache.flink.api.common.serialization.SimpleStringEncoder; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.DefaultRollingPolicy; + +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; + +/** + * Integration tests for {@link BucketAssigner bucket assigners}. + */ +public class BucketAssignerITCases { + + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Test + public void testAssembleBucketPath() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final Path basePath = new Path(outDir.toURI()); + final long time = 1000L; + + final RollingPolicy rollingPolicy = + DefaultRollingPolicy + .create() + .withMaxPartSize(7L) + .build(); + + final Buckets buckets = new Buckets<>( + basePath, + new BasePathBucketAssigner<>(), + new DefaultBucketFactoryImpl<>(), + new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), + rollingPolicy, + 0 + ); + + Bucket bucket = + buckets.onElement("abc", new TestUtils.MockSinkContext(time, time, time)); + Assert.assertEquals(new Path(basePath.toUri()), bucket.getBucketPath()); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java index aee362178a76e..8369b90ac9665 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java @@ -188,7 +188,7 @@ public void testOnProcessingTime() throws Exception { // it takes the current processing time of the context for the creation time, // and for the last modification time. - buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L , 3L)); + buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L, 3L)); // now it should roll buckets.onProcessingTime(7L); @@ -214,13 +214,13 @@ public void testBucketIsRemovedWhenNotActive() throws Exception { final Path path = new Path(outDir.toURI()); final OnProcessingTimePolicy rollOnProcessingTimeCountingPolicy = - new OnProcessingTimePolicy<>(2L); + new OnProcessingTimePolicy<>(2L); final Buckets buckets = createBuckets(path, rollOnProcessingTimeCountingPolicy, 0); // it takes the current processing time of the context for the creation time, and for the last modification time. - buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L , 3L)); + buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L, 3L)); // now it should roll buckets.onProcessingTime(7L); @@ -244,7 +244,7 @@ public void testPartCounterAfterBucketResurrection() throws Exception { createBuckets(path, rollOnProcessingTimeCountingPolicy, 0); // it takes the current processing time of the context for the creation time, and for the last modification time. - buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L , 3L)); + buckets.onElement("test", new TestUtils.MockSinkContext(1L, 2L, 3L)); Assert.assertEquals(1L, buckets.getActiveBuckets().get("test").getPartCounter()); // now it should roll @@ -257,7 +257,7 @@ public void testPartCounterAfterBucketResurrection() throws Exception { Assert.assertTrue(buckets.getActiveBuckets().isEmpty()); - buckets.onElement("test", new TestUtils.MockSinkContext(2L, 3L , 4L)); + buckets.onElement("test", new TestUtils.MockSinkContext(2L, 3L, 4L)); Assert.assertEquals(2L, buckets.getActiveBuckets().get("test").getPartCounter()); } From fb7a59d9ff76e4a1600af7fef83176c99f9ed860 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 9 Jan 2019 10:33:57 +0100 Subject: [PATCH 269/359] [FLINK-11262][py] Bump jython-standalone to 2.7.1 Bump the jython dependency because of a security issue. --- NOTICE-binary | 2 +- flink-libraries/flink-streaming-python/pom.xml | 2 +- .../flink-streaming-python/src/main/resources/META-INF/NOTICE | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/NOTICE-binary b/NOTICE-binary index 74df3d1c03ddd..0b10a27b9e958 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -5797,7 +5797,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Python Software Foundation License. (https://opensource.org/licenses/PythonSoftFoundation.php) See bundled license files for details. -- org.python:jython-standalone:2.7.0 +- org.python:jython-standalone:2.7.1 flink-metrics-graphite Copyright 2014-2018 The Apache Software Foundation diff --git a/flink-libraries/flink-streaming-python/pom.xml b/flink-libraries/flink-streaming-python/pom.xml index 212fcc09ff9ed..b25fa6d28967c 100644 --- a/flink-libraries/flink-streaming-python/pom.xml +++ b/flink-libraries/flink-streaming-python/pom.xml @@ -61,7 +61,7 @@ under the License. org.python jython-standalone - 2.7.0 + 2.7.1 diff --git a/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE b/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE index f89bbcbf77d06..6fcb19f3bc146 100644 --- a/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE +++ b/flink-libraries/flink-streaming-python/src/main/resources/META-INF/NOTICE @@ -7,4 +7,4 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Python Software Foundation License. (https://opensource.org/licenses/PythonSoftFoundation.php) See bundled license files for details. -- org.python:jython-standalone:2.7.0 +- org.python:jython-standalone:2.7.1 From d209ed356978d04bc0bd426877fae20bfb455b8c Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Wed, 9 Jan 2019 11:00:43 +0100 Subject: [PATCH 270/359] [FLINK-11207][build] Bump commons-compress to 1.18 This addresses CVE-2018-11771. --- NOTICE-binary | 6 +++--- flink-dist/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- pom.xml | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/NOTICE-binary b/NOTICE-binary index 0b10a27b9e958..2619dbab859a6 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -64,7 +64,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-collections:commons-collections:3.2.2 - commons-io:commons-io:2.4 - org.apache.camel:camel-core:2.17.7 -- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-compress:1.18 - org.apache.commons:commons-lang3:3.3.2 - org.apache.commons:commons-math3:3.5 - org.javassist:javassist:3.19.0-GA @@ -344,7 +344,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.htrace:htrace-core4:4.0.1-incubating - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-compress:1.18 - org.apache.commons:commons-math3:3.5 - commons-beanutils:commons-beanutils:1.8.3 - commons-cli:commons-cli:1.3.1 @@ -2412,7 +2412,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-logging:commons-logging:1.1.3 - commons-net:commons-net:3.1 - org.apache.avro:avro:1.8.2 -- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-compress:1.18 - org.apache.commons:commons-math3:3.5 - org.apache.zookeeper:zookeeper:3.4.10 - org.codehaus.jackson:jackson-core-asl:1.9.13 diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE index 7088b1ab775bc..442787a8b516d 100644 --- a/flink-dist/src/main/resources/META-INF/NOTICE +++ b/flink-dist/src/main/resources/META-INF/NOTICE @@ -20,7 +20,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-collections:commons-collections:3.2.2 - commons-io:commons-io:2.4 - org.apache.camel:camel-core:2.17.7 -- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-compress:1.18 - org.apache.commons:commons-lang3:3.3.2 - org.apache.commons:commons-math3:3.5 - org.javassist:javassist:3.19.0-GA diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE index 1d4bcc1601f62..9f2c6350cf3be 100644 --- a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -16,7 +16,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.htrace:htrace-core4:4.0.1-incubating - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-compress:1.18 - org.apache.commons:commons-math3:3.5 - commons-beanutils:commons-beanutils:1.8.3 - commons-cli:commons-cli:1.3.1 diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE index ad28a9bbcad64..f6d007d94d15f 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE +++ b/flink-shaded-hadoop/flink-shaded-hadoop2-uber/src/main/resources/META-INF/NOTICE @@ -19,7 +19,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-logging:commons-logging:1.1.3 - commons-net:commons-net:3.1 - org.apache.avro:avro:1.8.2 -- org.apache.commons:commons-compress:1.4.1 +- org.apache.commons:commons-compress:1.18 - org.apache.commons:commons-math3:3.5 - org.apache.zookeeper:zookeeper:3.4.10 - org.codehaus.jackson:jackson-core-asl:1.9.13 diff --git a/pom.xml b/pom.xml index f3fa4144dc9e8..b6e346a2d353d 100644 --- a/pom.xml +++ b/pom.xml @@ -429,7 +429,7 @@ under the License. org.apache.commons commons-compress - 1.4.1 + 1.18 From 89dc0b4c74a568d5e6aa5f07cbd75dc9883f5c5b Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 20 Dec 2018 14:18:46 +0100 Subject: [PATCH 271/359] [FLINK-11134][rest] Do not log stacktrace for handled exceptions --- .../flink/runtime/rest/handler/AbstractHandler.java | 9 ++++++--- .../flink/runtime/rest/handler/AbstractRestHandler.java | 6 +++++- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java index 5a1c371d5a671..3ca0bd38f49d3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java @@ -117,15 +117,13 @@ protected void respondAsLeader(ChannelHandlerContext ctx, RoutedRequest routedRe try { request = MAPPER.readValue("{}", untypedResponseMessageHeaders.getRequestClass()); } catch (JsonParseException | JsonMappingException je) { - log.error("Request did not conform to expected format.", je); - throw new RestHandlerException("Bad request received.", HttpResponseStatus.BAD_REQUEST, je); + throw new RestHandlerException("Bad request received. Request did not conform to expected format.", HttpResponseStatus.BAD_REQUEST, je); } } else { try { ByteBufInputStream in = new ByteBufInputStream(msgContent); request = MAPPER.readValue(in, untypedResponseMessageHeaders.getRequestClass()); } catch (JsonParseException | JsonMappingException je) { - log.error("Failed to read request.", je); throw new RestHandlerException( String.format("Request did not match expected format %s.", untypedResponseMessageHeaders.getRequestClass().getSimpleName()), HttpResponseStatus.BAD_REQUEST, @@ -165,6 +163,11 @@ protected void respondAsLeader(ChannelHandlerContext ctx, RoutedRequest routedRe }); } catch (RestHandlerException rhe) { inFlightRequestTracker.deregisterRequest(); + if (log.isDebugEnabled()) { + log.error("Exception occurred in REST handler.", rhe); + } else { + log.error("Exception occurred in REST handler: {}", rhe.getMessage()); + } HandlerUtils.sendErrorResponse( ctx, httpRequest, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java index 0397cb875f295..3d74a7b80b02e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java @@ -89,7 +89,11 @@ private Tuple2 errorResponse(Throwable throwab Throwable error = ExceptionUtils.stripCompletionException(throwable); if (error instanceof RestHandlerException) { final RestHandlerException rhe = (RestHandlerException) error; - log.error("Exception occurred in REST handler.", rhe); + if (log.isDebugEnabled()) { + log.error("Exception occurred in REST handler.", rhe); + } else { + log.error("Exception occurred in REST handler: {}", rhe.getMessage()); + } return Tuple2.of(new ErrorResponseBody(rhe.getMessage()), rhe.getHttpResponseStatus()); } else { log.error("Implementation error: Unhandled exception.", error); From be6d5735687bab721800954173c4c886c0dfa247 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 3 Jan 2019 17:29:56 +0100 Subject: [PATCH 272/359] [hotfix][rest] Centralize REST error logging --- .../runtime/rest/handler/AbstractHandler.java | 42 +++++++++++++------ .../rest/handler/AbstractRestHandler.java | 28 +------------ .../AbstractTaskManagerFileHandler.java | 25 ++++------- 3 files changed, 40 insertions(+), 55 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java index 3ca0bd38f49d3..d46bb13d4265d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException; @@ -50,6 +51,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Arrays; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -158,33 +160,49 @@ protected void respondAsLeader(ChannelHandlerContext ctx, RoutedRequest routedRe final FileUploads finalUploadedFiles = uploadedFiles; requestProcessingFuture .whenComplete((Void ignored, Throwable throwable) -> { - inFlightRequestTracker.deregisterRequest(); - cleanupFileUploads(finalUploadedFiles); + if (throwable != null) { + handleException(ExceptionUtils.stripCompletionException(throwable), ctx, httpRequest) + .whenComplete((Void ignored2, Throwable throwable2) -> finalizeRequestProcessing(finalUploadedFiles)); + } else { + finalizeRequestProcessing(finalUploadedFiles); + } }); - } catch (RestHandlerException rhe) { - inFlightRequestTracker.deregisterRequest(); + } catch (Throwable e) { + final FileUploads finalUploadedFiles = uploadedFiles; + handleException(e, ctx, httpRequest) + .whenComplete((Void ignored, Throwable throwable) -> finalizeRequestProcessing(finalUploadedFiles)); + } + } + + private void finalizeRequestProcessing(FileUploads uploadedFiles) { + inFlightRequestTracker.deregisterRequest(); + cleanupFileUploads(uploadedFiles); + } + + private CompletableFuture handleException(Throwable throwable, ChannelHandlerContext ctx, HttpRequest httpRequest) { + if (throwable instanceof RestHandlerException) { + RestHandlerException rhe = (RestHandlerException) throwable; if (log.isDebugEnabled()) { log.error("Exception occurred in REST handler.", rhe); } else { log.error("Exception occurred in REST handler: {}", rhe.getMessage()); } - HandlerUtils.sendErrorResponse( + return HandlerUtils.sendErrorResponse( ctx, httpRequest, new ErrorResponseBody(rhe.getMessage()), rhe.getHttpResponseStatus(), responseHeaders); - cleanupFileUploads(uploadedFiles); - } catch (Throwable e) { - inFlightRequestTracker.deregisterRequest(); - log.error("Request processing failed.", e); - HandlerUtils.sendErrorResponse( + } else { + log.error("Implementation error: Unhandled exception.", throwable); + String stackTrace = String.format("", + ExceptionUtils.stringifyException(throwable)); + return HandlerUtils.sendErrorResponse( ctx, httpRequest, - new ErrorResponseBody("Internal server error."), + new ErrorResponseBody(Arrays.asList("Internal server error.", stackTrace)), HttpResponseStatus.INTERNAL_SERVER_ERROR, responseHeaders); - cleanupFileUploads(uploadedFiles); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java index 3d74a7b80b02e..992e2c58d061c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java @@ -19,17 +19,14 @@ package org.apache.flink.runtime.rest.handler; import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rest.handler.util.HandlerUtils; -import org.apache.flink.runtime.rest.messages.ErrorResponseBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.messages.MessageParameters; import org.apache.flink.runtime.rest.messages.RequestBody; import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler; @@ -39,7 +36,6 @@ import javax.annotation.Nonnull; -import java.util.Arrays; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -80,29 +76,7 @@ protected CompletableFuture respondToRequest(ChannelHandlerContext ctx, Ht response = FutureUtils.completedExceptionally(e); } - return response.handle((resp, throwable) -> throwable != null ? - errorResponse(throwable) : Tuple2.of(resp, messageHeaders.getResponseStatusCode())) - .thenCompose(r -> HandlerUtils.sendResponse(ctx, httpRequest, r.f0, r.f1, responseHeaders)); - } - - private Tuple2 errorResponse(Throwable throwable) { - Throwable error = ExceptionUtils.stripCompletionException(throwable); - if (error instanceof RestHandlerException) { - final RestHandlerException rhe = (RestHandlerException) error; - if (log.isDebugEnabled()) { - log.error("Exception occurred in REST handler.", rhe); - } else { - log.error("Exception occurred in REST handler: {}", rhe.getMessage()); - } - return Tuple2.of(new ErrorResponseBody(rhe.getMessage()), rhe.getHttpResponseStatus()); - } else { - log.error("Implementation error: Unhandled exception.", error); - String stackTrace = String.format("", - ExceptionUtils.stringifyException(throwable)); - return Tuple2.of( - new ErrorResponseBody(Arrays.asList("Internal server error.", stackTrace)), - HttpResponseStatus.INTERNAL_SERVER_ERROR); - } + return response.thenAccept(resp -> HandlerUtils.sendResponse(ctx, httpRequest, resp, messageHeaders.getResponseStatusCode(), responseHeaders)); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java index 8a20868ce3733..1781fe2f24efa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java @@ -24,12 +24,11 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException; +import org.apache.flink.runtime.rest.NotFoundException; import org.apache.flink.runtime.rest.handler.AbstractHandler; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; -import org.apache.flink.runtime.rest.handler.util.HandlerUtils; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; -import org.apache.flink.runtime.rest.messages.ErrorResponseBody; import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters; @@ -75,7 +74,6 @@ import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION; import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus.OK; import static org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion.HTTP_1_1; @@ -160,23 +158,18 @@ protected CompletableFuture respondToRequest(ChannelHandlerContext ctx, Ht fileBlobKeys.invalidate(taskManagerId); final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable); - final ErrorResponseBody errorResponseBody; - final HttpResponseStatus httpResponseStatus; if (strippedThrowable instanceof UnknownTaskExecutorException) { - errorResponseBody = new ErrorResponseBody("Unknown TaskExecutor " + taskManagerId + '.'); - httpResponseStatus = HttpResponseStatus.NOT_FOUND; + throw new CompletionException( + new NotFoundException( + String.format("Failed to transfer file from TaskExecutor %s because it was unknown.", taskManagerId), + strippedThrowable)); } else { - errorResponseBody = new ErrorResponseBody("Internal server error: " + throwable.getMessage() + '.'); - httpResponseStatus = INTERNAL_SERVER_ERROR; + throw new CompletionException( + new FlinkException( + String.format("Failed to transfer file from TaskExecutor %s.", taskManagerId), + strippedThrowable)); } - - HandlerUtils.sendErrorResponse( - ctx, - httpRequest, - errorResponseBody, - httpResponseStatus, - responseHeaders); } }); } From 151ee434004a9c85ae3b93c995cc2be0b4430836 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 3 Jan 2019 17:30:53 +0100 Subject: [PATCH 273/359] [hotfix][rest] Remove unnecessary instanceof check --- .../handler/taskmanager/AbstractTaskManagerFileHandler.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java index 1781fe2f24efa..01d818bfaca9d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java @@ -124,11 +124,7 @@ protected CompletableFuture respondToRequest(ChannelHandlerContext ctx, Ht blobKeyFuture = fileBlobKeys.get(taskManagerId); } catch (ExecutionException e) { final Throwable cause = ExceptionUtils.stripExecutionException(e); - if (cause instanceof RestHandlerException) { - throw (RestHandlerException) cause; - } else { - throw new RestHandlerException("Could not retrieve file blob key future.", HttpResponseStatus.INTERNAL_SERVER_ERROR, e); - } + throw new RestHandlerException("Could not retrieve file blob key future.", HttpResponseStatus.INTERNAL_SERVER_ERROR, cause); } final CompletableFuture resultFuture = blobKeyFuture.thenAcceptAsync( From 8b9fd7fd0fe4794608d3f41b76daabfb4888dddd Mon Sep 17 00:00:00 2001 From: Shimin Yang Date: Thu, 6 Dec 2018 14:39:46 +0800 Subject: [PATCH 274/359] [FLINK-11084][datastream] Forbid using two consecutive split transformations This closes #7258 --- .../streaming/api/datastream/DataStream.java | 2 + .../streaming/api/datastream/SplitStream.java | 1 + .../api/graph/StreamGraphGenerator.java | 18 ++++ .../flink/streaming/api/DataStreamTest.java | 100 ++++++++++++++++-- .../streaming/api/scala/DataStream.scala | 6 ++ 5 files changed, 121 insertions(+), 6 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 271d9bec33f0e..a0af5f1a8c317 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -234,7 +234,9 @@ public final DataStream union(DataStream... streams) { * {@link org.apache.flink.streaming.api.collector.selector.OutputSelector} * for directing the tuples. * @return The {@link SplitStream} + * @deprecated Please use side ouput instead. */ + @Deprecated public SplitStream split(OutputSelector outputSelector) { return new SplitStream<>(this, clean(outputSelector)); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java index 0beae32435d4c..7f28dc7b497d6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java @@ -33,6 +33,7 @@ * @param The type of the elements in the Stream */ +@Deprecated @PublicEvolving public class SplitStream extends DataStream { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java index 2c4ae4a993dfa..82400036c3c83 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java @@ -258,6 +258,8 @@ private Collection transformSplit(SplitTransformation split) { StreamTransformation input = split.getInput(); Collection resultIds = transform(input); + validateSplitTransformation(input); + // the recursive transform call might have transformed this already if (alreadyTransformed.containsKey(split)) { return alreadyTransformed.get(split); @@ -643,4 +645,20 @@ private String determineSlotSharingGroup(String specifiedGroup, Collection void validateSplitTransformation(StreamTransformation input) { + if (input instanceof SelectTransformation || input instanceof SplitTransformation) { + throw new IllegalStateException("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); + } else if (input instanceof SideOutputTransformation) { + throw new IllegalStateException("Split after side-outputs are not supported. Splits are deprecated. Please use side-outputs."); + } else if (input instanceof UnionTransformation) { + for (StreamTransformation transformation : ((UnionTransformation) input).getInputs()) { + validateSplitTransformation(transformation); + } + } else if (input instanceof PartitionTransformation) { + validateSplitTransformation(((PartitionTransformation) input).getInput()); + } else { + return; + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index aa6774dab8fc9..215e5485648f4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -79,6 +79,7 @@ import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; import org.apache.flink.util.TestLogger; import org.hamcrest.core.StringStartsWith; @@ -952,12 +953,7 @@ public boolean filter(Integer value) throws Exception { fail(e.getMessage()); } - OutputSelector outputSelector = new OutputSelector() { - @Override - public Iterable select(Integer value) { - return null; - } - }; + OutputSelector outputSelector = new DummyOutputSelector<>(); SplitStream split = unionFilter.split(outputSelector); split.select("dummy").addSink(new DiscardingSink()); @@ -1091,6 +1087,91 @@ public void testChannelSelectors() { assertTrue(globalPartitioner instanceof GlobalPartitioner); } + ///////////////////////////////////////////////////////////// + // Split testing + ///////////////////////////////////////////////////////////// + + @Test + public void testConsecutiveSplitRejection() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource src = env.fromElements(0, 0); + + OutputSelector outputSelector = new DummyOutputSelector<>(); + + src.split(outputSelector).split(outputSelector).addSink(new DiscardingSink<>()); + + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); + + env.getStreamGraph(); + } + + @Test + public void testSplitAfterSideOutputRejection() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource src = env.fromElements(0, 0); + + OutputTag outputTag = new OutputTag("dummy"){}; + OutputSelector outputSelector = new DummyOutputSelector<>(); + + src.getSideOutput(outputTag).split(outputSelector).addSink(new DiscardingSink<>()); + + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Split after side-outputs are not supported. Splits are deprecated. Please use side-outputs."); + + env.getStreamGraph(); + } + + @Test + public void testSelectBetweenConsecutiveSplitRejection() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource src = env.fromElements(0, 0); + + OutputSelector outputSelector = new DummyOutputSelector<>(); + + src.split(outputSelector).select("dummy").split(outputSelector).addSink(new DiscardingSink<>()); + + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); + + env.getStreamGraph(); + } + + @Test + public void testUnionBetweenConsecutiveSplitRejection() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource src = env.fromElements(0, 0); + + OutputSelector outputSelector = new DummyOutputSelector<>(); + + src.split(outputSelector).select("dummy").union(src.map(x -> x)).split(outputSelector).addSink(new DiscardingSink<>()); + + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); + + env.getStreamGraph(); + } + + @Test + public void testKeybyBetweenConsecutiveSplitRejection() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource src = env.fromElements(0, 0); + + OutputSelector outputSelector = new DummyOutputSelector<>(); + + src.split(outputSelector).select("dummy").keyBy(x -> x).split(outputSelector).addSink(new DiscardingSink<>()); + + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); + + env.getStreamGraph(); + } + ///////////////////////////////////////////////////////////// // KeyBy testing ///////////////////////////////////////////////////////////// @@ -1427,4 +1508,11 @@ public int getI() { return i; } } + + private class DummyOutputSelector implements OutputSelector { + @Override + public Iterable select(Integer value) { + return null; + } + } } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 23d216549f836..15dca2c9c0752 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -915,13 +915,19 @@ class DataStream[T](stream: JavaStream[T]) { * Operator used for directing tuples to specific named outputs using an * OutputSelector. Calling this method on an operator creates a new * [[SplitStream]]. + * + * @deprecated Please use side output instead. */ + @deprecated def split(selector: OutputSelector[T]): SplitStream[T] = asScalaStream(stream.split(selector)) /** * Creates a new [[SplitStream]] that contains only the elements satisfying the * given output selector predicate. + * + * @deprecated Please use side output instead. */ + @deprecated def split(fun: T => TraversableOnce[String]): SplitStream[T] = { if (fun == null) { throw new NullPointerException("OutputSelector must not be null.") From 9529a2a6274a1a654b5e4c8c2a792afe5846105d Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Fri, 11 Jan 2019 17:20:02 +0800 Subject: [PATCH 275/359] [FLINK-11224][scala-shell] Log is missing in scala-shell Logging configuration was set only for scala-shell in yarn mode. This commit sets the configuration for local and remote mode in start-scala-shell.sh script as well. --- .../start-script/start-scala-shell.sh | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/flink-scala-shell/start-script/start-scala-shell.sh b/flink-scala-shell/start-script/start-scala-shell.sh index 033d505032229..cc6ebb02737f5 100644 --- a/flink-scala-shell/start-script/start-scala-shell.sh +++ b/flink-scala-shell/start-script/start-scala-shell.sh @@ -75,15 +75,26 @@ do fi done -log_setting="" +if [ "$FLINK_IDENT_STRING" = "" ]; then + FLINK_IDENT_STRING="$USER" +fi + +MODE=$1 +LOG=$FLINK_LOG_DIR/flink-$FLINK_IDENT_STRING-scala-shell-$MODE-$HOSTNAME.log -if [[ $1 = "yarn" ]] +if [[ ($MODE = "local") || ($MODE = "remote") ]] then -FLINK_CLASSPATH=$FLINK_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR -log=$FLINK_LOG_DIR/flink-$FLINK_IDENT_STRING-scala-shell-yarn-$HOSTNAME.log -log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4j-yarn-session.properties -Dlogback.configurationFile=file:"$FLINK_CONF_DIR"/logback-yarn.xml" + LOG4J_CONFIG=log4j.properties + LOGBACK_CONFIG=logback.xml +elif [[ $1 = "yarn" ]] +then + LOG4J_CONFIG=log4j-yarn-session.properties + LOGBACK_CONFIG=logback-yarn.xml + FLINK_CLASSPATH=$FLINK_CLASSPATH:$HADOOP_CLASSPATH:$HADOOP_CONF_DIR:$YARN_CONF_DIR fi +log_setting="-Dlog.file="$LOG" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/$LOG4J_CONFIG -Dlogback.configurationFile=file:"$FLINK_CONF_DIR"/$LOGBACK_CONFIG" + if ${EXTERNAL_LIB_FOUND} then java -Dscala.color -cp "$FLINK_CLASSPATH" $log_setting org.apache.flink.api.scala.FlinkShell $@ --addclasspath "$EXT_CLASSPATH" From eb7db3aed4e9651ed8624aa7797c5d9f656b2059 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 8 Jan 2019 13:06:50 +0100 Subject: [PATCH 276/359] [FLINK-10848] Remove container requests after successful container allocation This commit removes container requests after containers have been allocated. This prevents that we will request more and more containers from Yarn in case of a recovery. Since we cannot rely on the reported container Resource, we remove the container request by using the requested Resource. This is due Yarn's DefaultResourceCalculator which neglects the number of vCores when allocating containers. --- .../flink/yarn/YarnFlinkResourceManager.java | 73 ++++++++++--- .../flink/yarn/YarnResourceManager.java | 103 ++++++++++++------ .../src/main/resources/log4j.properties | 2 +- .../yarn/YarnFlinkResourceManagerTest.java | 25 ++--- .../flink/yarn/YarnResourceManagerTest.java | 19 +++- .../src/test/resources/log4j-test.properties | 2 +- 6 files changed, 151 insertions(+), 73 deletions(-) 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 8e686bbbe34c1..e8e55c3330361 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 @@ -41,15 +41,20 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.slf4j.Logger; +import javax.annotation.Nonnull; + import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -72,6 +77,8 @@ public class YarnFlinkResourceManager extends FlinkResourceManager containersInLaunch; @@ -314,6 +321,21 @@ protected void fatalError(String message, Throwable error) { @Override protected void requestNewWorkers(int numWorkers) { + final Resource capability = getContainerResource(); + + for (int i = 0; i < numWorkers; i++) { + numPendingContainerRequests++; + LOG.info("Requesting new TaskManager container with {} megabytes memory. Pending requests: {}", + capability.getMemory(), numPendingContainerRequests); + + resourceManagerClient.addContainerRequest(createContainerRequest(capability)); + } + + // make sure we transmit the request fast and receive fast news of granted allocations + resourceManagerClient.setHeartbeatInterval(FAST_YARN_HEARTBEAT_INTERVAL_MS); + } + + private Resource getContainerResource() { final long mem = taskManagerParameters.taskManagerTotalMemoryMB(); final int containerMemorySizeMB; @@ -325,25 +347,15 @@ protected void requestNewWorkers(int numWorkers) { mem, containerMemorySizeMB); } - for (int i = 0; i < numWorkers; i++) { - numPendingContainerRequests++; - LOG.info("Requesting new TaskManager container with {} megabytes memory. Pending requests: {}", - containerMemorySizeMB, numPendingContainerRequests); - - // Priority for worker containers - priorities are intra-application - Priority priority = Priority.newInstance(0); - - // Resource requirements for worker containers - int taskManagerSlots = taskManagerParameters.numSlots(); - int vcores = config.getInteger(YarnConfigOptions.VCORES, Math.max(taskManagerSlots, 1)); - Resource capability = Resource.newInstance(containerMemorySizeMB, vcores); - - resourceManagerClient.addContainerRequest( - new AMRMClient.ContainerRequest(capability, null, null, priority)); - } + // Resource requirements for worker containers + int taskManagerSlots = taskManagerParameters.numSlots(); + int vcores = config.getInteger(YarnConfigOptions.VCORES, Math.max(taskManagerSlots, 1)); + return Resource.newInstance(containerMemorySizeMB, vcores); + } - // make sure we transmit the request fast and receive fast news of granted allocations - resourceManagerClient.setHeartbeatInterval(FAST_YARN_HEARTBEAT_INTERVAL_MS); + @Nonnull + private AMRMClient.ContainerRequest createContainerRequest(Resource capability) { + return new AMRMClient.ContainerRequest(capability, null, null, RM_REQUEST_PRIORITY); } @Override @@ -434,7 +446,14 @@ private void containersAllocated(List containers) { final int numRequired = getDesignatedWorkerPoolSize(); final int numRegistered = getNumberOfStartedTaskManagers(); + final Collection pendingRequests = getPendingRequests(); + final Iterator pendingRequestsIterator = pendingRequests.iterator(); + for (Container container : containers) { + if (numPendingContainerRequests > 0) { + numPendingContainerRequests -= 1; + resourceManagerClient.removeContainerRequest(pendingRequestsIterator.next()); + } numPendingContainerRequests = Math.max(0, numPendingContainerRequests - 1); LOG.info("Received new container: {} - Remaining pending container requests: {}", container.getId(), numPendingContainerRequests); @@ -487,6 +506,24 @@ private void containersAllocated(List containers) { triggerCheckWorkers(); } + private Collection getPendingRequests() { + final List> matchingRequests = resourceManagerClient.getMatchingRequests(RM_REQUEST_PRIORITY, ResourceRequest.ANY, getContainerResource()); + + final Collection result; + + if (matchingRequests.isEmpty()) { + result = Collections.emptyList(); + } else { + result = new ArrayList<>(matchingRequests.get(0)); + } + + Preconditions.checkState( + result.size() == numPendingContainerRequests, + "The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests %s != Number RM's pending container requests %s.", result.size(), numPendingContainerRequests); + + return result; + } + /** * Invoked when the ResourceManager informs of completed containers. * Called via an actor message by the callback from the ResourceManager client. diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 6ff5cd6648711..609ab42f590f0 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; @@ -41,6 +42,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.hadoop.yarn.api.ApplicationConstants; @@ -52,15 +54,20 @@ import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -73,6 +80,7 @@ */ public class YarnResourceManager extends ResourceManager implements AMRMClientAsync.CallbackHandler { + private static final Priority RM_REQUEST_PRIORITY = Priority.newInstance(1); /** The process environment variables. */ private final Map env; @@ -119,6 +127,8 @@ public class YarnResourceManager extends ResourceManager impleme private final Collection slotsPerWorker; + private final Resource resource; + public YarnResourceManager( RpcService rpcService, String resourceManagerEndpointId, @@ -169,6 +179,7 @@ public YarnResourceManager( this.numberOfTaskSlots = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); this.defaultTaskManagerMemoryMB = ConfigurationUtils.getTaskManagerHeapMemory(flinkConfig).getMebiBytes(); this.defaultCpus = flinkConfig.getInteger(YarnConfigOptions.VCORES, numberOfTaskSlots); + this.resource = Resource.newInstance(defaultTaskManagerMemoryMB, defaultCpus); this.slotsPerWorker = createSlotsPerWorker(numberOfTaskSlots); } @@ -292,17 +303,19 @@ protected void internalDeregisterApplication( @Override public Collection startNewWorker(ResourceProfile resourceProfile) { - // Priority for worker containers - priorities are intra-application - //TODO: set priority according to the resource allocated - Priority priority = Priority.newInstance(generatePriority(resourceProfile)); - int mem = resourceProfile.getMemoryInMB() < 0 ? defaultTaskManagerMemoryMB : resourceProfile.getMemoryInMB(); - int vcore = resourceProfile.getCpuCores() < 1 ? defaultCpus : (int) resourceProfile.getCpuCores(); - Resource capability = Resource.newInstance(mem, vcore); - requestYarnContainer(capability, priority); + Preconditions.checkArgument( + ResourceProfile.UNKNOWN.equals(resourceProfile), + "The YarnResourceManager does not support custom ResourceProfiles yet. It assumes that all containers have the same resources."); + requestYarnContainer(); return slotsPerWorker; } + @VisibleForTesting + Resource getContainerResource() { + return resource; + } + @Override public boolean stopWorker(final YarnWorkerNode workerNode) { final Container container = workerNode.getContainer(); @@ -343,8 +356,7 @@ public void onContainersCompleted(final List statuses) { if (yarnWorkerNode != null) { // Container completed unexpectedly ~> start a new one - final Container container = yarnWorkerNode.getContainer(); - requestYarnContainerIfRequired(container.getResource(), yarnWorkerNode.getContainer().getPriority()); + requestYarnContainerIfRequired(); } // Eagerly close the connection with task manager. closeTaskManagerConnection(resourceId, new Exception(containerStatus.getDiagnostics())); @@ -356,6 +368,9 @@ public void onContainersCompleted(final List statuses) { @Override public void onContainersAllocated(List containers) { runAsync(() -> { + final Collection pendingRequests = getPendingRequests(); + final Iterator pendingRequestsIterator = pendingRequests.iterator(); + for (Container container : containers) { log.info( "Received new container: {} - Remaining pending container requests: {}", @@ -363,7 +378,7 @@ public void onContainersAllocated(List containers) { numPendingContainerRequests); if (numPendingContainerRequests > 0) { - numPendingContainerRequests--; + removeContainerRequest(pendingRequestsIterator.next()); final String containerIdStr = container.getId().toString(); final ResourceID resourceId = new ResourceID(containerIdStr); @@ -385,7 +400,7 @@ public void onContainersAllocated(List containers) { workerNodeMap.remove(resourceId); resourceManagerClient.releaseAssignedContainer(container.getId()); // and ask for a new one - requestYarnContainerIfRequired(container.getResource(), container.getPriority()); + requestYarnContainerIfRequired(); } } else { // return the excessive containers @@ -402,6 +417,36 @@ public void onContainersAllocated(List containers) { }); } + private void removeContainerRequest(AMRMClient.ContainerRequest pendingContainerRequest) { + numPendingContainerRequests--; + + log.info("Removing container request {}. Pending container requests {}.", pendingContainerRequest, numPendingContainerRequests); + + resourceManagerClient.removeContainerRequest(pendingContainerRequest); + } + + private Collection getPendingRequests() { + final List> matchingRequests = resourceManagerClient.getMatchingRequests( + RM_REQUEST_PRIORITY, + ResourceRequest.ANY, + getContainerResource()); + + final Collection matchingContainerRequests; + + if (matchingRequests.isEmpty()) { + matchingContainerRequests = Collections.emptyList(); + } else { + final Collection collection = matchingRequests.get(0); + matchingContainerRequests = new ArrayList<>(collection); + } + + Preconditions.checkState( + matchingContainerRequests.size() == numPendingContainerRequests, + "The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests %s != Number RM's pending container requests %s.", matchingContainerRequests.size(), numPendingContainerRequests); + + return matchingContainerRequests; + } + @Override public void onShutdownRequest() { shutDown(); @@ -456,17 +501,17 @@ private static Tuple2 parseHostPort(String address) { /** * Request new container if pending containers cannot satisfies pending slot requests. */ - private void requestYarnContainerIfRequired(Resource resource, Priority priority) { + private void requestYarnContainerIfRequired() { int requiredTaskManagerSlots = getNumberRequiredTaskManagerSlots(); int pendingTaskManagerSlots = numPendingContainerRequests * numberOfTaskSlots; if (requiredTaskManagerSlots > pendingTaskManagerSlots) { - requestYarnContainer(resource, priority); + requestYarnContainer(); } } - private void requestYarnContainer(Resource resource, Priority priority) { - resourceManagerClient.addContainerRequest(new AMRMClient.ContainerRequest(resource, null, null, priority)); + private void requestYarnContainer() { + resourceManagerClient.addContainerRequest(getContainerRequest()); // make sure we transmit the request fast and receive fast news of granted allocations resourceManagerClient.setHeartbeatInterval(FAST_YARN_HEARTBEAT_INTERVAL_MS); @@ -478,6 +523,16 @@ private void requestYarnContainer(Resource resource, Priority priority) { numPendingContainerRequests); } + @Nonnull + @VisibleForTesting + AMRMClient.ContainerRequest getContainerRequest() { + return new AMRMClient.ContainerRequest( + getContainerResource(), + null, + null, + RM_REQUEST_PRIORITY); + } + private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource, String containerId, String host) throws Exception { // init the ContainerLaunchContext @@ -514,22 +569,4 @@ private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource .put(ENV_FLINK_NODE_ID, host); return taskExecutorLaunchContext; } - - - - /** - * Generate priority by given resource profile. - * Priority is only used for distinguishing request of different resource. - * @param resourceProfile The resource profile of a request - * @return The priority of this resource profile. - */ - private int generatePriority(ResourceProfile resourceProfile) { - if (resourcePriorities.containsKey(resourceProfile)) { - return resourcePriorities.get(resourceProfile); - } else { - int priority = resourcePriorities.size(); - resourcePriorities.put(resourceProfile, priority); - return priority; - } - } } diff --git a/flink-yarn/src/main/resources/log4j.properties b/flink-yarn/src/main/resources/log4j.properties index b2ad0d3c465bc..e84cd49f01aea 100644 --- a/flink-yarn/src/main/resources/log4j.properties +++ b/flink-yarn/src/main/resources/log4j.properties @@ -17,7 +17,7 @@ ################################################################################ # Convenience file for local debugging of the JobManager/TaskManager. -log4j.rootLogger=INFO, console +log4j.rootLogger=OFF, console log4j.appender.console=org.apache.log4j.ConsoleAppender log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java index 10b2ce97d6fe4..4108249548e56 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFlinkResourceManagerTest.java @@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; @@ -52,7 +54,6 @@ import org.junit.Test; import org.mockito.Matchers; import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.util.ArrayList; import java.util.Collections; @@ -69,7 +70,10 @@ import scala.concurrent.duration.FiniteDuration; import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -128,20 +132,6 @@ public void testYarnFlinkResourceManagerJobManagerLostLeadership() throws Except containerList.add(mockContainer); } - doAnswer(new Answer() { - int counter = 0; - @Override - public Object answer(InvocationOnMock invocation) throws Throwable { - if (counter < containerList.size()) { - callbackHandler.onContainersAllocated( - Collections.singletonList( - containerList.get(counter++) - )); - } - return null; - } - }).when(resourceManagerClient).addContainerRequest(Matchers.any(AMRMClient.ContainerRequest.class)); - final CompletableFuture resourceManagerFuture = new CompletableFuture<>(); final CompletableFuture leaderGatewayFuture = new CompletableFuture<>(); @@ -191,6 +181,9 @@ public Object answer(InvocationOnMock invocation) throws Throwable { nodeManagerClient )); + doReturn(Collections.singletonList(Collections.nCopies(numInitialTaskManagers, new AMRMClient.ContainerRequest(Resource.newInstance(1024 * 1024, 1), null, null, Priority.newInstance(0))))) + .when(resourceManagerClient).getMatchingRequests(any(Priority.class), anyString(), any(Resource.class)); + leaderRetrievalService.notifyListener(leader1.path().toString(), leaderSessionID); final AkkaActorGateway leader1Gateway = new AkkaActorGateway(leader1, leaderSessionID); @@ -203,6 +196,8 @@ public Object answer(InvocationOnMock invocation) throws Throwable { resourceManagerGateway.tell(new RegisterResourceManagerSuccessful(leader1, Collections.emptyList())); + callbackHandler.onContainersAllocated(containerList); + for (int i = 0; i < containerList.size(); i++) { expectMsgClass(deadline.timeLeft(), Acknowledge.class); } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java index d41d42d7a05f9..368e95cc76030 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java @@ -105,8 +105,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -243,7 +245,7 @@ class Context { final HardwareDescription hardwareDescription = new HardwareDescription(1, 2L, 3L, 4L); // domain objects for test purposes - final ResourceProfile resourceProfile1 = new ResourceProfile(1.0, 200); + final ResourceProfile resourceProfile1 = ResourceProfile.UNKNOWN; public ContainerId task = ContainerId.newInstance( ApplicationAttemptId.newInstance(ApplicationId.newInstance(1L, 0), 0), 1); @@ -351,7 +353,7 @@ void runTest(RunnableWithException testMethod) throws Exception { } } - private static Container mockContainer(String host, int port, int containerId) { + private static Container mockContainer(String host, int port, int containerId, Resource resource) { Container mockContainer = mock(Container.class); NodeId mockNodeId = NodeId.newInstance(host, port); @@ -365,7 +367,7 @@ private static Container mockContainer(String host, int port, int containerId) { when(mockContainer.getId()).thenReturn(mockContainerId); when(mockContainer.getNodeId()).thenReturn(mockNodeId); - when(mockContainer.getResource()).thenReturn(Resource.newInstance(200, 1)); + when(mockContainer.getResource()).thenReturn(resource); when(mockContainer.getPriority()).thenReturn(Priority.UNDEFINED); return mockContainer; @@ -397,7 +399,10 @@ public void testStopWorker() throws Exception { registerSlotRequestFuture.get(); // Callback from YARN when container is allocated. - Container testingContainer = mockContainer("container", 1234, 1); + Container testingContainer = mockContainer("container", 1234, 1, resourceManager.getContainerResource()); + + doReturn(Collections.singletonList(Collections.singletonList(resourceManager.getContainerRequest()))) + .when(mockResourceManagerClient).getMatchingRequests(any(Priority.class), anyString(), any(Resource.class)); resourceManager.onContainersAllocated(ImmutableList.of(testingContainer)); verify(mockResourceManagerClient).addContainerRequest(any(AMRMClient.ContainerRequest.class)); @@ -492,10 +497,14 @@ public void testOnContainerCompleted() throws Exception { registerSlotRequestFuture.get(); // Callback from YARN when container is allocated. - Container testingContainer = mockContainer("container", 1234, 1); + Container testingContainer = mockContainer("container", 1234, 1, resourceManager.getContainerResource()); + + doReturn(Collections.singletonList(Collections.singletonList(resourceManager.getContainerRequest()))) + .when(mockResourceManagerClient).getMatchingRequests(any(Priority.class), anyString(), any(Resource.class)); resourceManager.onContainersAllocated(ImmutableList.of(testingContainer)); verify(mockResourceManagerClient).addContainerRequest(any(AMRMClient.ContainerRequest.class)); + verify(mockResourceManagerClient).removeContainerRequest(any(AMRMClient.ContainerRequest.class)); verify(mockNMClient).startContainer(eq(testingContainer), any(ContainerLaunchContext.class)); // Callback from YARN when container is Completed, pending request can not be fulfilled by pending diff --git a/flink-yarn/src/test/resources/log4j-test.properties b/flink-yarn/src/test/resources/log4j-test.properties index 2226f68653181..5b1e4ed8e7a74 100644 --- a/flink-yarn/src/test/resources/log4j-test.properties +++ b/flink-yarn/src/test/resources/log4j-test.properties @@ -18,7 +18,7 @@ # Set root logger level to OFF to not flood build logs # set manually to INFO for debugging purposes -log4j.rootLogger=OFF, testlogger +log4j.rootLogger=INFO, testlogger # A1 is set to be a ConsoleAppender. log4j.appender.testlogger=org.apache.log4j.ConsoleAppender From 233a0321a02ac992ac54fb1632ee564451853124 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Mon, 14 Jan 2019 09:25:42 +0800 Subject: [PATCH 277/359] [hotfix] [docs] Fix kafka doc mistake classname --- docs/dev/connectors/kafka.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 351a4dc2d4126..e769faa7a465f 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -547,7 +547,7 @@ In the meanwhile, a possible workaround is to send *heartbeat messages* to all c ## Kafka Producer -Flink’s Kafka Producer is called `FlinkKafkaProducer011` (or `010` for Kafka 0.10.0.x versions, etc. or just `FlinkKafkaConsumer` for Kafka >= 1.0.0 versions). +Flink’s Kafka Producer is called `FlinkKafkaProducer011` (or `010` for Kafka 0.10.0.x versions, etc. or just `FlinkKafkaProducer` for Kafka >= 1.0.0 versions). It allows writing a stream of records to one or more Kafka topics. Example: From f6da7617a0c036421393ae5cdaa9a289d36e6883 Mon Sep 17 00:00:00 2001 From: JaryZhen Date: Mon, 14 Jan 2019 12:30:10 +0800 Subject: [PATCH 278/359] [FLINK-11304][docs][table] Fix typo in time attributes doc This closes #7477. --- docs/dev/table/streaming/time_attributes.md | 24 +++++++++++++++------ 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/docs/dev/table/streaming/time_attributes.md b/docs/dev/table/streaming/time_attributes.md index 27208fb768ddb..447b058c05c23 100644 --- a/docs/dev/table/streaming/time_attributes.md +++ b/docs/dev/table/streaming/time_attributes.md @@ -264,7 +264,7 @@ Moreover, the `DataStream` returned by the `getDataStream()` method must have wa
    {% highlight java %} // define a table source with a rowtime attribute -public class UserActionSource implements StreamTableSource, DefinedRowtimeAttribute { +public class UserActionSource implements StreamTableSource, DefinedRowtimeAttributes { @Override public TypeInformation getReturnType() { @@ -284,9 +284,15 @@ public class UserActionSource implements StreamTableSource, DefinedRowtimeA } @Override - public String getRowtimeAttribute() { + public List getRowtimeAttributeDescriptors() { // Mark the "UserActionTime" attribute as event-time attribute. - return "UserActionTime"; + // We create one attribute descriptor of "UserActionTime". + RowtimeAttributeDescriptor rowtimeAttrDescr = new RowtimeAttributeDescriptor( + "UserActionTime", + new ExistingField("UserActionTime"), + new AscendingTimestamps()); + List listRowtimeAttrDescr = Collections.singletonList(rowtimeAttrDescr); + return listRowtimeAttrDescr; } } @@ -301,7 +307,7 @@ WindowedTable windowedTable = tEnv
    {% highlight scala %} // define a table source with a rowtime attribute -class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttribute { +class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttributes { override def getReturnType = { val names = Array[String]("Username" , "Data", "UserActionTime") @@ -317,9 +323,15 @@ class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttribu stream } - override def getRowtimeAttribute = { + override def getRowtimeAttributeDescriptors: util.List[RowtimeAttributeDescriptor] = { // Mark the "UserActionTime" attribute as event-time attribute. - "UserActionTime" + // We create one attribute descriptor of "UserActionTime". + val rowtimeAttrDescr = new RowtimeAttributeDescriptor( + "UserActionTime", + new ExistingField("UserActionTime"), + new AscendingTimestamps) + val listRowtimeAttrDescr = Collections.singletonList(rowtimeAttrDescr) + listRowtimeAttrDescr } } From 21019c36c853b24dca2be4b584163ff0db9cd74d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 14 Jan 2019 14:05:46 +0100 Subject: [PATCH 279/359] [hotfix][docs][table] Fix more typos in time attributes doc --- docs/dev/table/streaming/time_attributes.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/streaming/time_attributes.md b/docs/dev/table/streaming/time_attributes.md index 447b058c05c23..64da17189751a 100644 --- a/docs/dev/table/streaming/time_attributes.md +++ b/docs/dev/table/streaming/time_attributes.md @@ -256,9 +256,12 @@ val windowedTable = table.window(Tumble over 10.minutes on 'UserActionTime as 'u ### Using a TableSource -The event time attribute is defined by a `TableSource` that implements the `DefinedRowtimeAttribute` interface. The `getRowtimeAttribute()` method returns the name of an existing field that carries the event time attribute of the table and is of type `LONG` or `TIMESTAMP`. +The event time attribute is defined by a `TableSource` that implements the `DefinedRowtimeAttributes` interface. The `getRowtimeAttributeDescriptors()` method returns a list of `RowtimeAttributeDescriptor` for describing the final name of a time attribute, a timestamp extractor to derive the values of the attribute, and the watermark strategy associated with the attribute. -Moreover, the `DataStream` returned by the `getDataStream()` method must have watermarks assigned that are aligned with the defined time attribute. Please note that the timestamps of the `DataStream` (the ones which are assigned by a `TimestampAssigner`) are ignored. Only the values of the `TableSource`'s rowtime attribute are relevant. +Please make sure that the `DataStream` returned by the `getDataStream()` method is aligned with the defined time attribute. +The timestamps of the `DataStream` (the ones which are assigned by a `TimestampAssigner`) are only considered if a `StreamRecordTimestamp` timestamp extractor is defined. +Watermarks of a `DataStream` are only preserved if a `PreserveWatermarks` watermark strategy is defined. +Otherwise, only the values of the `TableSource`'s rowtime attribute are relevant.
    From aa728c53281b35aecd37d9a158ad61f1015f7a28 Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Tue, 15 Jan 2019 18:31:57 +0800 Subject: [PATCH 280/359] [FLINK-11331][table][docs] Fix errors in tableApi.md and functions.md This closes #7494 --- docs/dev/table/functions.md | 52 ++++++++++++++++++++++++++++++++++++- docs/dev/table/tableApi.md | 4 +-- 2 files changed, 53 insertions(+), 3 deletions(-) diff --git a/docs/dev/table/functions.md b/docs/dev/table/functions.md index 682f519eb3ce6..f41139d14957c 100644 --- a/docs/dev/table/functions.md +++ b/docs/dev/table/functions.md @@ -3573,7 +3573,18 @@ NUMERIC.years

    Creates an interval of months for NUMERIC years.

    - + + + {% highlight java %} +NUMERIC.quarter +NUMERIC.quarters +{% endhighlight %} + + +

    Creates an interval of months for NUMERIC quarters.

    +

    E.g., 2.quarters returns 6.

    + + {% highlight java %} @@ -3586,6 +3597,19 @@ NUMERIC.months + + + {% highlight java %} +NUMERIC.week +NUMERIC.weeks +{% endhighlight %} + + +

    Creates an interval of milliseconds for NUMERIC weeks.

    +

    E.g., 2.weeks returns 1209600000.

    + + + {% highlight java %} @@ -3831,6 +3855,19 @@ NUMERIC.years + + + {% highlight scala %} +NUMERIC.quarter +NUMERIC.quarters +{% endhighlight %} + + +

    Creates an interval of months for NUMERIC quarters.

    +

    E.g., 2.quarters returns 6.

    + + + {% highlight scala %} @@ -3843,6 +3880,19 @@ NUMERIC.months + + + {% highlight scala %} +NUMERIC.week +NUMERIC.weeks +{% endhighlight %} + + +

    Creates an interval of milliseconds for NUMERIC weeks.

    +

    E.g., 2.weeks returns 1209600000.

    + + + {% highlight scala %} diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index 45ee5a4e17d71..9bd720c69a2c2 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -109,7 +109,7 @@ Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) Table result = orders .filter("a.isNotNull && b.isNotNull && c.isNotNull") - .select("a.lowerCase(), b, rowtime") + .select("a.lowerCase() as a, b, rowtime") .window(Tumble.over("1.hour").on("rowtime").as("hourlyWindow")) .groupBy("hourlyWindow, a") .select("a, hourlyWindow.end as hour, b.avg as avgBillingAmount"); @@ -128,7 +128,7 @@ val orders: Table = tEnv.scan("Orders") // schema (a, b, c, rowtime) val result: Table = orders .filter('a.isNotNull && 'b.isNotNull && 'c.isNotNull) - .select('a.lowerCase(), 'b, 'rowtime) + .select('a.lowerCase() as 'a, 'b, 'rowtime) .window(Tumble over 1.hour on 'rowtime as 'hourlyWindow) .groupBy('hourlyWindow, 'a) .select('a, 'hourlyWindow.end as 'hour, 'b.avg as 'avgBillingAmount) From cbf258aa890aff118b2697169316ea9dee9d5643 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 29 Nov 2018 12:03:05 +0100 Subject: [PATCH 281/359] [FLINK-11023] Add LICENSE & NOTICE files for flink-connector-cassandra --- .../src/main/resources/META-INF/NOTICE | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE diff --git a/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..a0449ff300703 --- /dev/null +++ b/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE @@ -0,0 +1,16 @@ +flink-connector-cassandra +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.datastax.cassandra:cassandra-driver-core:3.0.0 +- com.datastax.cassandra:cassandra-driver-mapping:3.0.0 +- com.google.guava:guava:18.0 +- io.netty:netty-handler:4.0.33.Final +- io.netty:netty-buffer:4.0.33.Final +- io.netty:netty-common:4.0.33.Final +- io.netty:netty-transport:4.0.33.Final +- io.netty:netty-codec:4.0.33.Final From c4f1198a7396259bfd56d5843e99643bb92b3acc Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 29 Nov 2018 12:03:31 +0100 Subject: [PATCH 282/359] [FLINK-11023] Add LICENSE & NOTICE files for flink-connector-elasticsearch --- .../src/main/resources/META-INF/NOTICE | 133 ++++-------------- .../resources/META-INF/licenses/LICENSE.antlr | 29 ++++ 2 files changed, 55 insertions(+), 107 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/licenses/LICENSE.antlr diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE index f588e0e873c14..a58d21ae6c6b0 100644 --- a/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/NOTICE @@ -1,109 +1,28 @@ -This project includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -------------------------------------------------------------- - -This project bundles the following dependencies under -the Apache Software License 2.0 - - - org.apache.lucene : lucene-core version 4.10.4 - - org.apache.lucene : lucene-analyzers-common version 4.10.4 - - org.apache.lucene : lucene-grouping version 4.10.4 - - org.apache.lucene : lucene-highlighter version 4.10.4 - - org.apache.lucene : lucene-join version 4.10.4 - - org.apache.lucene : lucene-memory version 4.10.4 - - org.apache.lucene : lucene-misc version 4.10.4 - - org.apache.lucene : lucene-queries version 4.10.4 - - org.apache.lucene : lucene-queryparser version 4.10.4 - - org.apache.lucene : lucene-sandbox version 4.10.4 - - org.apache.lucene : lucene-spatial version 4.10.4 - - org.apache.lucene : lucene-suggest version 4.10.4 - - com.spatial4j : spatial4j version 0.4.1 - - com.fasterxml.jackson.core : jackson-core version 2.5.3 - - com.fasterxml.jackson.dataformat : jackson-dataformat-smile version 2.5.3 - - com.fasterxml.jackson.dataformat : jackson-dataformat-yaml version 2.5.3 - - com.fasterxml.jackson.dataformat : jackson-dataformat-cbor version 2.5.3 - - org.joda : joda-convert (copied classes) - -=================================== - Notice for Yaml -=================================== - -This project bundles yaml (v. 1.12) under the Creative Commons License (CC-BY 2.0). - -Original project website: http://www.yaml.de - -Copyright (c) 2005-2013, Dirk Jesse - -YAML under Creative Commons License (CC-BY 2.0) -=============================================== - -The YAML framework is published under the Creative Commons Attribution 2.0 License (CC-BY 2.0), which permits -both private and commercial use (http://creativecommons.org/licenses/by/2.0/). - -Condition: For the free use of the YAML framework, a backlink to the YAML homepage (http://www.yaml.de) in a -suitable place (e.g.: footer of the website or in the imprint) is required. - -=================================== - Notice for Tartarus -=================================== - -This project bundles tartarus under the MIT License. +flink-connector-elasticsearch +Copyright 2014-2018 The Apache Software Foundation -Original source repository: https://github.com/sergiooramas/tartarus - -Copyright (c) 2017 Sergio Oramas and Oriol Nieto - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -SOFTWARE. - -=================================== - Notice for Antlr -=================================== - -This project bundles antlr-runtime (v. 3.5) under the BSD 2-Clause License - -Copyright (c) 2010 Terence Parr - ------------------------------------------------------------------------------ -** Beginning of "BSD 2-Clause License" text. ** - - Copyright (c) 2010 Terence Parr - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are met: - - 1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.spatial4j:spatial4j:0.4.1 +- org.apache.lucene:lucene-analyzers-common:4.10.4 +- org.apache.lucene:lucene-core:4.10.4 +- org.apache.lucene:lucene-grouping:4.10.4 +- org.apache.lucene:lucene-highlighter:4.10.4 +- org.apache.lucene:lucene-join:4.10.4 +- org.apache.lucene:lucene-memory:4.10.4 +- org.apache.lucene:lucene-misc:4.10.4 +- org.apache.lucene:lucene-queries:4.10.4 +- org.apache.lucene:lucene-queryparser:4.10.4 +- org.apache.lucene:lucene-sandbox:4.10.4 +- org.apache.lucene:lucene-spatial:4.10.4 +- org.apache.lucene:lucene-suggest:4.10.4 +- org.elasticsearch:elasticsearch:1.7.1 +- org.yaml:snakeyaml:1.12 + +This project bundles the following dependencies under the BSD License. +See bundled licenses for details. + +- org.antlr:antlr-runtime:jar:3.5 diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/licenses/LICENSE.antlr b/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/licenses/LICENSE.antlr new file mode 100644 index 0000000000000..6d149604e134b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch/src/main/resources/META-INF/licenses/LICENSE.antlr @@ -0,0 +1,29 @@ +Copyright (c) 2010 Terence Parr + +----------------------------------------------------------------------------- +** Beginning of "BSD 2-Clause License" text. ** + + Copyright (c) 2010 Terence Parr + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. From 1cd253b3ba1ab3c3c8f4febcbf96de1a9255bfba Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 29 Nov 2018 12:03:54 +0100 Subject: [PATCH 283/359] [FLINK-11023] Add LICENSE & NOTICE files for flink-connector-elasticsearch2 +es2 --- .../flink-connector-elasticsearch2/pom.xml | 18 ++ .../src/main/resources/META-INF/NOTICE | 212 +++++++----------- .../META-INF/licenses/LICENSE.base64 | 26 +++ .../META-INF/licenses/LICENSE.hdrhistogram | 125 +++++++++++ .../META-INF/licenses/LICENSE.jsr166e | 121 ++++++++++ .../META-INF/licenses/LICENSE.jsr166y | 26 +++ .../resources/META-INF/licenses/LICENSE.jzlib | 26 +++ .../META-INF/licenses/LICENSE.webbit | 38 ++++ 8 files changed, 461 insertions(+), 131 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.base64 create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166e create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166y create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jzlib create mode 100644 flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.webbit diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml index 29fbf3ea33874..73d9e977758da 100644 --- a/flink-connectors/flink-connector-elasticsearch2/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml @@ -194,6 +194,24 @@ under the License. META-INF/maven/org.y*/** + + + *:* + + META-INF/LICENSE.txt + + + + io.netty:netty + + + META-INF/license + + META-INF/NOTICE.txt + + diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE index c3d95ca99c016..4288bc1b48174 100644 --- a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/NOTICE @@ -1,133 +1,83 @@ -This project includes software developed at +flink-connector-elasticsearch2 +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -------------------------------------------------------------- - -This project bundles the following dependencies under -the Apache Software License 2.0 - - - org.apache.lucene : lucene-core version 5.5.0 - - org.apache.lucene : lucene-analyzers-common version 5.5.0 - - org.apache.lucene : lucene-backward-codecs version 5.5.0 - - org.apache.lucene : lucene-grouping version 5.5.0 - - org.apache.lucene : lucene-highlighter version 5.5.0 - - org.apache.lucene : lucene-join version 5.5.0 - - org.apache.lucene : lucene-memory version 5.5.0 - - org.apache.lucene : lucene-misc version 5.5.0 - - org.apache.lucene : lucene-queries version 5.5.0 - - org.apache.lucene : lucene-queryparser version 5.5.0 - - org.apache.lucene : lucene-sandbox version 5.5.0 - - org.apache.lucene : lucene-spatial version 5.5.0 - - org.apache.lucene : lucene-spatial-extras version 5.5.0 - - org.apache.lucene : lucene-spatial3d version 5.5.0 - - org.apache.lucene : lucene-suggest version 5.5.0 - - com.carrotsearch : hppc version 0.7.1 - - com.google.guava : guava version 18.0 - - com.google.code.findbugs : jsr305 version 1.3.9 - - com.ning : compress-lzf version 1.0.2 - - com.spatial4j : spatial4j version 0.5 - - com.twitter : chill-java version 0.7.4 - - com.fasterxml.jackson.core : jackson-core version 2.6.6 - - com.fasterxml.jackson.dataformat : jackson-dataformat-smile version 2.6.6 - - com.fasterxml.jackson.dataformat : jackson-dataformat-yaml version 2.6.6 - - com.fasterxml.jackson.dataformat : jackson-dataformat-cbor version 2.6.6 - - com.tdunning : t-digest version 3.0 - - io.netty : netty version 3.10.5.Final - - joda-time : joda-time version 2.5 - -=================================== - Notice for Yaml -=================================== - -This project bundles yaml (v. 1.15) under the Creative Commons License (CC-BY 2.0). - -Original project website: http://www.yaml.de - -Copyright (c) 2005-2013, Dirk Jesse - -YAML under Creative Commons License (CC-BY 2.0) -=============================================== - -The YAML framework is published under the Creative Commons Attribution 2.0 License (CC-BY 2.0), which permits -both private and commercial use (http://creativecommons.org/licenses/by/2.0/). - -Condition: For the free use of the YAML framework, a backlink to the YAML homepage (http://www.yaml.de) in a -suitable place (e.g.: footer of the website or in the imprint) is required. - -=================================== - Notice for Tartarus -=================================== - -This project bundles tartarus under the MIT License. - -Original source repository: https://github.com/sergiooramas/tartarus - -Copyright (c) 2017 Sergio Oramas and Oriol Nieto - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -SOFTWARE. - -=================================== - Notice for HdrHistogram -=================================== - -This project bundles HdrHistogram (v. 2.1.9) under the BSD 2-Clause License - -Original source repository: https://github.com/HdrHistogram/HdrHistogram - -The code in this repository code was Written by Gil Tene, Michael Barker, -and Matt Warren, and released to the public domain, as explained at -http://creativecommons.org/publicdomain/zero/1.0/ - -For users of this code who wish to consume it under the "BSD" license -rather than under the public domain or CC0 contribution text mentioned -above, the code found under this directory is *also* provided under the -following license (commonly referred to as the BSD 2-Clause License). This -license does not detract from the above stated release of the code into -the public domain, and simply represents an additional license granted by -the Author. - ------------------------------------------------------------------------------ -** Beginning of "BSD 2-Clause License" text. ** - - Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene - Copyright (c) 2014 Michael Barker - Copyright (c) 2014 Matt Warren - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are met: - - 1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.carrotsearch:hppc:0.7.1 +- com.fasterxml.jackson.core:jackson-core:2.6.6 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.6.6 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.6.6 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.6.6 +- com.google.guava:guava:18.0 +- com.ning:compress-lzf:1.0.2 +- com.spatial4j:spatial4j:0.5 +- com.tdunning:t-digest:3.0 +- commons-cli:commons-cli:1.3.1 +- io.netty:netty:3.10.5.Final +- joda-time:joda-time:2.5 +- org.apache.lucene:lucene-analyzers-common:5.5.0 +- org.apache.lucene:lucene-backward-codecs:5.5.0 +- org.apache.lucene:lucene-core:5.5.0 +- org.apache.lucene:lucene-grouping:5.5.0 +- org.apache.lucene:lucene-highlighter:5.5.0 +- org.apache.lucene:lucene-join:5.5.0 +- org.apache.lucene:lucene-memory:5.5.0 +- org.apache.lucene:lucene-misc:5.5.0 +- org.apache.lucene:lucene-queries:5.5.0 +- org.apache.lucene:lucene-queryparser:5.5.0 +- org.apache.lucene:lucene-sandbox:5.5.0 +- org.apache.lucene:lucene-spatial:5.5.0 +- org.apache.lucene:lucene-spatial3d:5.5.0 +- org.apache.lucene:lucene-suggest:5.5.0 +- org.elasticsearch:elasticsearch:2.3.5 +- org.elasticsearch:securesm:1.0 +- org.joda:joda-convert:1.7 +- org.yaml:snakeyaml:1.15 + +This project bundles the following dependencies under the Creative Commons CC0 1.0 Universal Public Domain Dedication License (http://creativecommons.org/publicdomain/zero/1.0/) +See bundled license files for details. + +- org.hdrhistogram:HdrHistogram:2.1.6 + +This project bundles the following dependency under the CC0 1.0. +See bundled licenses for details. + +- com.twitter:jsr166e:1.1.0 + +This project bundles io.netty:netty:3.10.5.Final from which it inherits the following notices: + +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * licenses/LICENSE.jsr166y (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.base64 (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.jzlib (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product contains a modified version of 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * licenses/LICENSE.webbit (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.base64 b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.base64 new file mode 100644 index 0000000000000..31ebc840539c1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.base64 @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuate of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram new file mode 100644 index 0000000000000..09c38ea51608e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram @@ -0,0 +1,125 @@ +The code was Written by Gil Tene, Michael Barker, and Matt Warren, +and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ + +Creative Commons Legal Code + +CC0 1.0 Universal + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS + PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM + THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED + HEREUNDER. + +Statement of Purpose + +The laws of most jurisdictions throughout the world automatically confer +exclusive Copyright and Related Rights (defined below) upon the creator +and subsequent owner(s) (each and all, an "owner") of an original work of +authorship and/or a database (each, a "Work"). + +Certain owners wish to permanently relinquish those rights to a Work for +the purpose of contributing to a commons of creative, cultural and +scientific works ("Commons") that the public can reliably and without fear +of later claims of infringement build upon, modify, incorporate in other +works, reuse and redistribute as freely as possible in any form whatsoever +and for any purposes, including without limitation commercial purposes. +These owners may contribute to the Commons to promote the ideal of a free +culture and the further production of creative, cultural and scientific +works, or to gain reputation or greater distribution for their Work in +part through the use and efforts of others. + +For these and/or other purposes and motivations, and without any +expectation of additional consideration or compensation, the person +associating CC0 with a Work (the "Affirmer"), to the extent that he or she +is an owner of Copyright and Related Rights in the Work, voluntarily +elects to apply CC0 to the Work and publicly distribute the Work under its +terms, with knowledge of his or her Copyright and Related Rights in the +Work and the meaning and intended legal effect of CC0 on those rights. + +1. Copyright and Related Rights. A Work made available under CC0 may be +protected by copyright and related or neighboring rights ("Copyright and +Related Rights"). Copyright and Related Rights include, but are not +limited to, the following: + + i. the right to reproduce, adapt, distribute, perform, display, + communicate, and translate a Work; + ii. moral rights retained by the original author(s) and/or performer(s); +iii. publicity and privacy rights pertaining to a person's image or + likeness depicted in a Work; + iv. rights protecting against unfair competition in regards to a Work, + subject to the limitations in paragraph 4(a), below; + v. rights protecting the extraction, dissemination, use and reuse of data + in a Work; + vi. database rights (such as those arising under Directive 96/9/EC of the + European Parliament and of the Council of 11 March 1996 on the legal + protection of databases, and under any national implementation + thereof, including any amended or successor version of such + directive); and +vii. other similar, equivalent or corresponding rights throughout the + world based on applicable law or treaty, and any national + implementations thereof. + +2. Waiver. To the greatest extent permitted by, but not in contravention +of, applicable law, Affirmer hereby overtly, fully, permanently, +irrevocably and unconditionally waives, abandons, and surrenders all of +Affirmer's Copyright and Related Rights and associated claims and causes +of action, whether now known or unknown (including existing as well as +future claims and causes of action), in the Work (i) in all territories +worldwide, (ii) for the maximum duration provided by applicable law or +treaty (including future time extensions), (iii) in any current or future +medium and for any number of copies, and (iv) for any purpose whatsoever, +including without limitation commercial, advertising or promotional +purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each +member of the public at large and to the detriment of Affirmer's heirs and +successors, fully intending that such Waiver shall not be subject to +revocation, rescission, cancellation, termination, or any other legal or +equitable action to disrupt the quiet enjoyment of the Work by the public +as contemplated by Affirmer's express Statement of Purpose. + +3. Public License Fallback. Should any part of the Waiver for any reason +be judged legally invalid or ineffective under applicable law, then the +Waiver shall be preserved to the maximum extent permitted taking into +account Affirmer's express Statement of Purpose. In addition, to the +extent the Waiver is so judged Affirmer hereby grants to each affected +person a royalty-free, non transferable, non sublicensable, non exclusive, +irrevocable and unconditional license to exercise Affirmer's Copyright and +Related Rights in the Work (i) in all territories worldwide, (ii) for the +maximum duration provided by applicable law or treaty (including future +time extensions), (iii) in any current or future medium and for any number +of copies, and (iv) for any purpose whatsoever, including without +limitation commercial, advertising or promotional purposes (the +"License"). The License shall be deemed effective as of the date CC0 was +applied by Affirmer to the Work. Should any part of the License for any +reason be judged legally invalid or ineffective under applicable law, such +partial invalidity or ineffectiveness shall not invalidate the remainder +of the License, and in such case Affirmer hereby affirms that he or she +will not (i) exercise any of his or her remaining Copyright and Related +Rights in the Work or (ii) assert any associated claims and causes of +action with respect to the Work, in either case contrary to Affirmer's +express Statement of Purpose. + +4. Limitations and Disclaimers. + + a. No trademark or patent rights held by Affirmer are waived, abandoned, + surrendered, licensed or otherwise affected by this document. + b. Affirmer offers the Work as-is and makes no representations or + warranties of any kind concerning the Work, express, implied, + statutory or otherwise, including without limitation warranties of + title, merchantability, fitness for a particular purpose, non + infringement, or the absence of latent or other defects, accuracy, or + the present or absence of errors, whether or not discoverable, all to + the greatest extent permissible under applicable law. + c. Affirmer disclaims responsibility for clearing rights of other persons + that may apply to the Work or any use thereof, including without + limitation any person's Copyright and Related Rights in the Work. + Further, Affirmer disclaims responsibility for obtaining any necessary + consents, permissions or other rights required for any use of the + Work. + d. Affirmer understands and acknowledges that Creative Commons is not a + party to this document and has no duty or obligation with respect to + this CC0 or use of the Work. diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166e b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166e new file mode 100644 index 0000000000000..0e259d42c9967 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166e @@ -0,0 +1,121 @@ +Creative Commons Legal Code + +CC0 1.0 Universal + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS + PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM + THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED + HEREUNDER. + +Statement of Purpose + +The laws of most jurisdictions throughout the world automatically confer +exclusive Copyright and Related Rights (defined below) upon the creator +and subsequent owner(s) (each and all, an "owner") of an original work of +authorship and/or a database (each, a "Work"). + +Certain owners wish to permanently relinquish those rights to a Work for +the purpose of contributing to a commons of creative, cultural and +scientific works ("Commons") that the public can reliably and without fear +of later claims of infringement build upon, modify, incorporate in other +works, reuse and redistribute as freely as possible in any form whatsoever +and for any purposes, including without limitation commercial purposes. +These owners may contribute to the Commons to promote the ideal of a free +culture and the further production of creative, cultural and scientific +works, or to gain reputation or greater distribution for their Work in +part through the use and efforts of others. + +For these and/or other purposes and motivations, and without any +expectation of additional consideration or compensation, the person +associating CC0 with a Work (the "Affirmer"), to the extent that he or she +is an owner of Copyright and Related Rights in the Work, voluntarily +elects to apply CC0 to the Work and publicly distribute the Work under its +terms, with knowledge of his or her Copyright and Related Rights in the +Work and the meaning and intended legal effect of CC0 on those rights. + +1. Copyright and Related Rights. A Work made available under CC0 may be +protected by copyright and related or neighboring rights ("Copyright and +Related Rights"). Copyright and Related Rights include, but are not +limited to, the following: + + i. the right to reproduce, adapt, distribute, perform, display, + communicate, and translate a Work; + ii. moral rights retained by the original author(s) and/or performer(s); +iii. publicity and privacy rights pertaining to a person's image or + likeness depicted in a Work; + iv. rights protecting against unfair competition in regards to a Work, + subject to the limitations in paragraph 4(a), below; + v. rights protecting the extraction, dissemination, use and reuse of data + in a Work; + vi. database rights (such as those arising under Directive 96/9/EC of the + European Parliament and of the Council of 11 March 1996 on the legal + protection of databases, and under any national implementation + thereof, including any amended or successor version of such + directive); and +vii. other similar, equivalent or corresponding rights throughout the + world based on applicable law or treaty, and any national + implementations thereof. + +2. Waiver. To the greatest extent permitted by, but not in contravention +of, applicable law, Affirmer hereby overtly, fully, permanently, +irrevocably and unconditionally waives, abandons, and surrenders all of +Affirmer's Copyright and Related Rights and associated claims and causes +of action, whether now known or unknown (including existing as well as +future claims and causes of action), in the Work (i) in all territories +worldwide, (ii) for the maximum duration provided by applicable law or +treaty (including future time extensions), (iii) in any current or future +medium and for any number of copies, and (iv) for any purpose whatsoever, +including without limitation commercial, advertising or promotional +purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each +member of the public at large and to the detriment of Affirmer's heirs and +successors, fully intending that such Waiver shall not be subject to +revocation, rescission, cancellation, termination, or any other legal or +equitable action to disrupt the quiet enjoyment of the Work by the public +as contemplated by Affirmer's express Statement of Purpose. + +3. Public License Fallback. Should any part of the Waiver for any reason +be judged legally invalid or ineffective under applicable law, then the +Waiver shall be preserved to the maximum extent permitted taking into +account Affirmer's express Statement of Purpose. In addition, to the +extent the Waiver is so judged Affirmer hereby grants to each affected +person a royalty-free, non transferable, non sublicensable, non exclusive, +irrevocable and unconditional license to exercise Affirmer's Copyright and +Related Rights in the Work (i) in all territories worldwide, (ii) for the +maximum duration provided by applicable law or treaty (including future +time extensions), (iii) in any current or future medium and for any number +of copies, and (iv) for any purpose whatsoever, including without +limitation commercial, advertising or promotional purposes (the +"License"). The License shall be deemed effective as of the date CC0 was +applied by Affirmer to the Work. Should any part of the License for any +reason be judged legally invalid or ineffective under applicable law, such +partial invalidity or ineffectiveness shall not invalidate the remainder +of the License, and in such case Affirmer hereby affirms that he or she +will not (i) exercise any of his or her remaining Copyright and Related +Rights in the Work or (ii) assert any associated claims and causes of +action with respect to the Work, in either case contrary to Affirmer's +express Statement of Purpose. + +4. Limitations and Disclaimers. + + a. No trademark or patent rights held by Affirmer are waived, abandoned, + surrendered, licensed or otherwise affected by this document. + b. Affirmer offers the Work as-is and makes no representations or + warranties of any kind concerning the Work, express, implied, + statutory or otherwise, including without limitation warranties of + title, merchantability, fitness for a particular purpose, non + infringement, or the absence of latent or other defects, accuracy, or + the present or absence of errors, whether or not discoverable, all to + the greatest extent permissible under applicable law. + c. Affirmer disclaims responsibility for clearing rights of other persons + that may apply to the Work or any use thereof, including without + limitation any person's Copyright and Related Rights in the Work. + Further, Affirmer disclaims responsibility for obtaining any necessary + consents, permissions or other rights required for any use of the + Work. + d. Affirmer understands and acknowledges that Creative Commons is not a + party to this document and has no duty or obligation with respect to + this CC0 or use of the Work. diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166y b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166y new file mode 100644 index 0000000000000..b1c292b54cb2b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jsr166y @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuity of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jzlib b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jzlib new file mode 100644 index 0000000000000..29ad562af0a5e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.jzlib @@ -0,0 +1,26 @@ +Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.webbit b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.webbit new file mode 100644 index 0000000000000..ec5f348998bd4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch2/src/main/resources/META-INF/licenses/LICENSE.webbit @@ -0,0 +1,38 @@ +(BSD License: http://www.opensource.org/licenses/bsd-license) + +Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the +following conditions are met: + +* Redistributions of source code must retain the above + copyright notice, this list of conditions and the + following disclaimer. + +* Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the + following disclaimer in the documentation and/or other + materials provided with the distribution. + +* Neither the name of the Webbit nor the names of + its contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND +CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE +GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT +OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + From e5f052c1124227ec12afd3904fc28de8aa6d8dc7 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 29 Nov 2018 12:04:24 +0100 Subject: [PATCH 284/359] [FLINK-11023] Add LICENSE & NOTICE files for flink-connector-elasticsearch5 +es5 --- .../flink-connector-elasticsearch5/pom.xml | 18 ++ .../src/main/resources/META-INF/NOTICE | 297 ++++++------------ .../META-INF/licenses/LICENSE.base64 | 26 ++ .../META-INF/licenses/LICENSE.hdrhistogram | 125 ++++++++ .../META-INF/licenses/LICENSE.joptsimple | 22 ++ .../META-INF/licenses/LICENSE.jsr166y | 26 ++ .../resources/META-INF/licenses/LICENSE.jzlib | 26 ++ .../META-INF/licenses/LICENSE.webbit | 38 +++ 8 files changed, 381 insertions(+), 197 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.base64 create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.joptsimple create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jsr166y create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jzlib create mode 100644 flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.webbit diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml index a8a6489f22c4a..4ec8f8b770945 100644 --- a/flink-connectors/flink-connector-elasticsearch5/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml @@ -231,6 +231,13 @@ under the License. + + + *:* + + META-INF/LICENSE.txt + + * @@ -263,6 +270,17 @@ under the License. META-INF/log4j-provider.properties + + io.netty:netty + + + META-INF/license + + META-INF/NOTICE.txt + + diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE index 19e96faa46003..9fe894149aed0 100644 --- a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE @@ -1,201 +1,104 @@ -This project includes software developed at -The Apache Software Foundation (http://www.apache.org/). +flink-connector-elasticsearch5 +Copyright 2014-2018 The Apache Software Foundation -------------------------------------------------------------- - -This project bundles the following dependencies under -the Apache Software License 2.0 - - - org.apache.lucene : lucene-core version 6.3.0 - - org.apache.lucene : lucene-analyzers-common version 6.3.0 - - org.apache.lucene : lucene-backward-codecs version 6.3.0 - - org.apache.lucene : lucene-grouping version 6.3.0 - - org.apache.lucene : lucene-highlighter version 6.3.0 - - org.apache.lucene : lucene-join version 6.3.0 - - org.apache.lucene : lucene-memory version 6.3.0 - - org.apache.lucene : lucene-misc version 6.3.0 - - org.apache.lucene : lucene-queries version 6.3.0 - - org.apache.lucene : lucene-queryparser version 6.3.0 - - org.apache.lucene : lucene-sandbox version 6.3.0 - - org.apache.lucene : lucene-spatial version 6.3.0 - - org.apache.lucene : lucene-spatial-extras version 6.3.0 - - org.apache.lucene : lucene-spatial3d version 6.3.0 - - org.apache.lucene : lucene-suggest version 6.3.0 - - org.apache.httpcomponents : httpclient version 4.5.3 - - org.apache.httpcomponents : httpcore version 4.4.6 - - org.apache.httpcomponents : httpasynclcient version 4.1.2 - - org.apache.httpcomponents : httpcore-nio version 4.4.5 - - com.carrotsearch : hppc version 0.7.1 - - com.fasterxml.jackson.core : jackson-core version 2.8.1 - - com.fasterxml.jackson.dataformat : jackson-dataformat-smile version 2.8.1 - - com.fasterxml.jackson.dataformat : jackson-dataformat-yaml version 2.8.1 - - com.fasterxml.jackson.dataformat : jackson-dataformat-cbor version 2.8.1 - - com.tdunning : t-digest version 3.0 - - io.netty : netty version 3.10.6.Final - - io.netty : netty-buffer version 4.1.6.Final - - io.netty : netty-codec version 4.1.6.Final - - io.netty : netty-codec-http version 4.1.6.Final - - io.netty : netty-common version 4.1.6.Final - - io.netty : netty-handler version 4.1.6.Final - - io.netty : netty-resolver version 4.1.6.Final - - io.netty : netty-transport version 4.1.6.Final - - org.jboss.netty : netty version 3.2.0.Final - - joda-time : joda-time version 2.5 - - com.github.spullara.mustache.java : compiler version 0.9.3 - -=================================== - Notice for Yaml -=================================== - -This project bundles yaml (v. 1.15) under the Creative Commons License (CC-BY 2.0). - -Original project website: http://www.yaml.de - -Copyright (c) 2005-2013, Dirk Jesse - -YAML under Creative Commons License (CC-BY 2.0) -=============================================== - -The YAML framework is published under the Creative Commons Attribution 2.0 License (CC-BY 2.0), which permits -both private and commercial use (http://creativecommons.org/licenses/by/2.0/). - -Condition: For the free use of the YAML framework, a backlink to the YAML homepage (http://www.yaml.de) in a -suitable place (e.g.: footer of the website or in the imprint) is required. - -=================================== - Notice for Tartarus -=================================== - -This project bundles tartarus under the MIT License. - -Original source repository: https://github.com/sergiooramas/tartarus - -Copyright (c) 2017 Sergio Oramas and Oriol Nieto - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE -SOFTWARE. - -=================================== - Notice for joptsimple -=================================== - -This project bundles joptsimple under the MIT License. - -Original source repository: https://github.com/jopt-simple/jopt-simple - -Copyright (c) 2004-2016 Paul R. Holser, Jr. - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -"Software"), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: - -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND -NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE -LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION -OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION -WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - -=================================== - Notice for scopt -=================================== - -This project bundles scopt (v. 3.5.0) underr the MIT License. - -Original source repository: https://github.com/scopt/scopt - -scopt - Copyright (c) scopt contributors - -See source files for details. - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -``Software''), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.carrotsearch:hppc:0.7.1 +- com.fasterxml.jackson.core:jackson-core:2.8.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.1 +- com.github.spullara.mustache.java:compiler:0.9.3 +- com.tdunning:t-digest:3.0 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- io.netty:netty:3.10.6.Final +- io.netty:netty-buffer:4.1.6.Final +- io.netty:netty-codec:4.1.6.Final +- io.netty:netty-codec-http:4.1.6.Final +- io.netty:netty-common:4.1.6.Final +- io.netty:netty-handler:4.1.6.Final +- io.netty:netty-resolver:4.1.6.Final +- io.netty:netty-transport:4.1.6.Final +- joda.time:joda-time:2.5 +- net.java.dev.jna:jna:4.2.2 +- org.apache.httpcomponents:httpasynclcient:4.1.2 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpcore-nio:4.4.5 +- org.apache.logging.log4j:log4j-api:2.7 +- org.apache.logging.log4j:log4j-to-slf4j:2.7 +- org.apache.lucene:lucene-analyzers-common:6.3.0 +- org.apache.lucene:lucene-backward-codecs:6.3.0 +- org.apache.lucene:lucene-core:6.3.0 +- org.apache.lucene:lucene-grouping:6.3.0 +- org.apache.lucene:lucene-highlighter:6.3.0 +- org.apache.lucene:lucene-join:6.3.0 +- org.apache.lucene:lucene-memory:6.3.0 +- org.apache.lucene:lucene-misc:6.3.0 +- org.apache.lucene:lucene-queries:6.3.0 +- org.apache.lucene:lucene-queryparser:6.3.0 +- org.apache.lucene:lucene-sandbox:6.3.0 +- org.apache.lucene:lucene-spatial:6.3.0 +- org.apache.lucene:lucene-spatial-extras:6.3.0 +- org.apache.lucene:lucene-spatial3d:6.3.0 +- org.apache.lucene:lucene-suggest:6.3.0 +- org.elasticsearch:elasticsearch:5.1.2 +- org.elasticsearch:securesm:1.1 +- org.elasticsearch.client:rest:5.1.2 +- org.elasticsearch.client:transport:5.1.2 +- org.elasticsearch.plugin:lang-mustache-client:5.1.2 +- org.elasticsearch.plugin:percolator-client:5.1.2 +- org.elasticsearch.plugin:reindex-client:5.1.2 +- org.elasticsearch.plugin:transport-netty3-client:5.1.2 +- org.elasticsearch.plugin:transport-netty4-client:5.1.2 +- org.yaml:snakeyaml:1.15 + +This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) + +- net.sf.jopt-simple:jopt-simple:5.0.2 + +This project bundles the following dependencies under the Creative Commons CC0 1.0 Universal Public Domain Dedication License (http://creativecommons.org/publicdomain/zero/1.0/) +See bundled license files for details. + +- org.hdrhistogram:HdrHistogram:2.1.6 + +This project bundles io.netty:netty:3.10.6.Final from which it inherits the following notices: + +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * licenses/LICENSE.jsr166y (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.base64 (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * licenses/LICENSE.jzlib (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product contains a modified version of 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * licenses/LICENSE.webbit (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit -THE SOFTWARE IS PROVIDED ``AS IS'', WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. -IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY -CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, -TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE -SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -=================================== - Notice for HdrHistogram -=================================== - -This project bundles HdrHistogram (v. 2.1.9) under the BSD 2-Clause License - -Original source repository: https://github.com/HdrHistogram/HdrHistogram - -The code in this repository code was Written by Gil Tene, Michael Barker, -and Matt Warren, and released to the public domain, as explained at -http://creativecommons.org/publicdomain/zero/1.0/ - -For users of this code who wish to consume it under the "BSD" license -rather than under the public domain or CC0 contribution text mentioned -above, the code found under this directory is *also* provided under the -following license (commonly referred to as the BSD 2-Clause License). This -license does not detract from the above stated release of the code into -the public domain, and simply represents an additional license granted by -the Author. - ------------------------------------------------------------------------------ -** Beginning of "BSD 2-Clause License" text. ** - - Copyright (c) 2012, 2013, 2014, 2015, 2016 Gil Tene - Copyright (c) 2014 Michael Barker - Copyright (c) 2014 Matt Warren - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions are met: - - 1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.base64 b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.base64 new file mode 100644 index 0000000000000..31ebc840539c1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.base64 @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuate of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram new file mode 100644 index 0000000000000..09c38ea51608e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.hdrhistogram @@ -0,0 +1,125 @@ +The code was Written by Gil Tene, Michael Barker, and Matt Warren, +and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ + +Creative Commons Legal Code + +CC0 1.0 Universal + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS + PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM + THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED + HEREUNDER. + +Statement of Purpose + +The laws of most jurisdictions throughout the world automatically confer +exclusive Copyright and Related Rights (defined below) upon the creator +and subsequent owner(s) (each and all, an "owner") of an original work of +authorship and/or a database (each, a "Work"). + +Certain owners wish to permanently relinquish those rights to a Work for +the purpose of contributing to a commons of creative, cultural and +scientific works ("Commons") that the public can reliably and without fear +of later claims of infringement build upon, modify, incorporate in other +works, reuse and redistribute as freely as possible in any form whatsoever +and for any purposes, including without limitation commercial purposes. +These owners may contribute to the Commons to promote the ideal of a free +culture and the further production of creative, cultural and scientific +works, or to gain reputation or greater distribution for their Work in +part through the use and efforts of others. + +For these and/or other purposes and motivations, and without any +expectation of additional consideration or compensation, the person +associating CC0 with a Work (the "Affirmer"), to the extent that he or she +is an owner of Copyright and Related Rights in the Work, voluntarily +elects to apply CC0 to the Work and publicly distribute the Work under its +terms, with knowledge of his or her Copyright and Related Rights in the +Work and the meaning and intended legal effect of CC0 on those rights. + +1. Copyright and Related Rights. A Work made available under CC0 may be +protected by copyright and related or neighboring rights ("Copyright and +Related Rights"). Copyright and Related Rights include, but are not +limited to, the following: + + i. the right to reproduce, adapt, distribute, perform, display, + communicate, and translate a Work; + ii. moral rights retained by the original author(s) and/or performer(s); +iii. publicity and privacy rights pertaining to a person's image or + likeness depicted in a Work; + iv. rights protecting against unfair competition in regards to a Work, + subject to the limitations in paragraph 4(a), below; + v. rights protecting the extraction, dissemination, use and reuse of data + in a Work; + vi. database rights (such as those arising under Directive 96/9/EC of the + European Parliament and of the Council of 11 March 1996 on the legal + protection of databases, and under any national implementation + thereof, including any amended or successor version of such + directive); and +vii. other similar, equivalent or corresponding rights throughout the + world based on applicable law or treaty, and any national + implementations thereof. + +2. Waiver. To the greatest extent permitted by, but not in contravention +of, applicable law, Affirmer hereby overtly, fully, permanently, +irrevocably and unconditionally waives, abandons, and surrenders all of +Affirmer's Copyright and Related Rights and associated claims and causes +of action, whether now known or unknown (including existing as well as +future claims and causes of action), in the Work (i) in all territories +worldwide, (ii) for the maximum duration provided by applicable law or +treaty (including future time extensions), (iii) in any current or future +medium and for any number of copies, and (iv) for any purpose whatsoever, +including without limitation commercial, advertising or promotional +purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each +member of the public at large and to the detriment of Affirmer's heirs and +successors, fully intending that such Waiver shall not be subject to +revocation, rescission, cancellation, termination, or any other legal or +equitable action to disrupt the quiet enjoyment of the Work by the public +as contemplated by Affirmer's express Statement of Purpose. + +3. Public License Fallback. Should any part of the Waiver for any reason +be judged legally invalid or ineffective under applicable law, then the +Waiver shall be preserved to the maximum extent permitted taking into +account Affirmer's express Statement of Purpose. In addition, to the +extent the Waiver is so judged Affirmer hereby grants to each affected +person a royalty-free, non transferable, non sublicensable, non exclusive, +irrevocable and unconditional license to exercise Affirmer's Copyright and +Related Rights in the Work (i) in all territories worldwide, (ii) for the +maximum duration provided by applicable law or treaty (including future +time extensions), (iii) in any current or future medium and for any number +of copies, and (iv) for any purpose whatsoever, including without +limitation commercial, advertising or promotional purposes (the +"License"). The License shall be deemed effective as of the date CC0 was +applied by Affirmer to the Work. Should any part of the License for any +reason be judged legally invalid or ineffective under applicable law, such +partial invalidity or ineffectiveness shall not invalidate the remainder +of the License, and in such case Affirmer hereby affirms that he or she +will not (i) exercise any of his or her remaining Copyright and Related +Rights in the Work or (ii) assert any associated claims and causes of +action with respect to the Work, in either case contrary to Affirmer's +express Statement of Purpose. + +4. Limitations and Disclaimers. + + a. No trademark or patent rights held by Affirmer are waived, abandoned, + surrendered, licensed or otherwise affected by this document. + b. Affirmer offers the Work as-is and makes no representations or + warranties of any kind concerning the Work, express, implied, + statutory or otherwise, including without limitation warranties of + title, merchantability, fitness for a particular purpose, non + infringement, or the absence of latent or other defects, accuracy, or + the present or absence of errors, whether or not discoverable, all to + the greatest extent permissible under applicable law. + c. Affirmer disclaims responsibility for clearing rights of other persons + that may apply to the Work or any use thereof, including without + limitation any person's Copyright and Related Rights in the Work. + Further, Affirmer disclaims responsibility for obtaining any necessary + consents, permissions or other rights required for any use of the + Work. + d. Affirmer understands and acknowledges that Creative Commons is not a + party to this document and has no duty or obligation with respect to + this CC0 or use of the Work. diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.joptsimple b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.joptsimple new file mode 100644 index 0000000000000..ef77fc95e107e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.joptsimple @@ -0,0 +1,22 @@ +The MIT License + +Copyright (c) 2004-2015 Paul R. Holser, Jr. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jsr166y b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jsr166y new file mode 100644 index 0000000000000..b1c292b54cb2b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jsr166y @@ -0,0 +1,26 @@ +The person or persons who have associated work with this document (the +"Dedicator" or "Certifier") hereby either (a) certifies that, to the best of +his knowledge, the work of authorship identified is in the public domain of +the country from which the work is published, or (b) hereby dedicates whatever +copyright the dedicators holds in the work of authorship identified below (the +"Work") to the public domain. A certifier, moreover, dedicates any copyright +interest he may have in the associated work, and for these purposes, is +described as a "dedicator" below. + +A certifier has taken reasonable steps to verify the copyright status of this +work. Certifier recognizes that his good faith efforts may not shield him from +liability if in fact the work certified is not in the public domain. + +Dedicator makes this dedication for the benefit of the public at large and to +the detriment of the Dedicator's heirs and successors. Dedicator intends this +dedication to be an overt act of relinquishment in perpetuity of all present +and future rights under copyright law, whether vested or contingent, in the +Work. Dedicator understands that such relinquishment of all rights includes +the relinquishment of all rights to enforce (by lawsuit or otherwise) those +copyrights in the Work. + +Dedicator recognizes that, once placed in the public domain, the Work may be +freely reproduced, distributed, transmitted, used, modified, built upon, or +otherwise exploited by anyone for any purpose, commercial or non-commercial, +and in any way, including by methods that have not yet been invented or +conceived. diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jzlib b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jzlib new file mode 100644 index 0000000000000..29ad562af0a5e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.jzlib @@ -0,0 +1,26 @@ +Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + 3. The names of the authors may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, +INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.webbit b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.webbit new file mode 100644 index 0000000000000..ec5f348998bd4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/licenses/LICENSE.webbit @@ -0,0 +1,38 @@ +(BSD License: http://www.opensource.org/licenses/bsd-license) + +Copyright (c) 2011, Joe Walnes, Aslak Hellesøy and contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the +following conditions are met: + +* Redistributions of source code must retain the above + copyright notice, this list of conditions and the + following disclaimer. + +* Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the + following disclaimer in the documentation and/or other + materials provided with the distribution. + +* Neither the name of the Webbit nor the names of + its contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND +CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE +GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR +BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT +OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + From b61ab11d1939c591c65e9288747c2b88fdde838c Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 29 Nov 2018 12:04:42 +0100 Subject: [PATCH 285/359] [FLINK-11023] Add LICENSE & NOTICE files for flink-connector-kinesis --- .../src/main/resources/META-INF/NOTICE | 26 +++++++++++++ .../META-INF/licenses/LICENSE.amazon | 39 +++++++++++++++++++ .../META-INF/licenses/LICENSE.protobuf | 32 +++++++++++++++ 3 files changed, 97 insertions(+) create mode 100644 flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.amazon create mode 100644 flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.protobuf diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..a7dcc98cbadd0 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -0,0 +1,26 @@ +flink-connector-kinesis +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.amazonaws:aws-java-sdk-core:1.11.319 +- com.amazonaws:aws-java-sdk-kinesis:1.11.319 +- com.amazonaws:aws-java-sdk-sts:1.11.319 +- com.amazonaws:jmespath-java:1.11.319 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- com.google.protobuf:protobuf-java:2.6.1 + +This project bundles the following dependencies under the Amazon Softward License. +See bundled license files for details. + +- com.amazonaws:amazon-kinesis-client:1.9.0 +- com.amazonaws:amazon-kinesis-producer:0.12.9 + diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.amazon b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.amazon new file mode 100644 index 0000000000000..e933f6953c1ca --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.amazon @@ -0,0 +1,39 @@ +Amazon Software License + +This Amazon Software License (“License”) governs your use, reproduction, and distribution of the accompanying software as specified below. +1. Definitions + +“Licensor” means any person or entity that distributes its Work. + +“Software” means the original work of authorship made available under this License. + +“Work” means the Software and any additions to or derivative works of the Software that are made available under this License. + +The terms “reproduce,” “reproduction,” “derivative works,” and “distribution” have the meaning as provided under U.S. copyright law; provided, however, that for the purposes of this License, derivative works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work. + +Works, including the Software, are “made available” under this License by including in or with the Work either (a) a copyright notice referencing the applicability of this License to the Work, or (b) a copy of this License. +2. License Grants + +2.1 Copyright Grant. Subject to the terms and conditions of this License, each Licensor grants to you a perpetual, worldwide, non-exclusive, royalty-free, copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, sublicense and distribute its Work and any resulting derivative works in any form. + +2.2 Patent Grant. Subject to the terms and conditions of this License, each Licensor grants to you a perpetual, worldwide, non-exclusive, royalty-free patent license to make, have made, use, sell, offer for sale, import, and otherwise transfer its Work, in whole or in part. The foregoing license applies only to the patent claims licensable by Licensor that would be infringed by Licensor’s Work (or portion thereof) individually and excluding any combinations with any other materials or technology. +3. Limitations + +3.1 Redistribution. You may reproduce or distribute the Work only if (a) you do so under this License, (b) you include a complete copy of this License with your distribution, and (c) you retain without modification any copyright, patent, trademark, or attribution notices that are present in the Work. + +3.2 Derivative Works. You may specify that additional or different terms apply to the use, reproduction, and distribution of your derivative works of the Work (“Your Terms”) only if (a) Your Terms provide that the use limitation in Section 3.3 applies to your derivative works, and (b) you identify the specific derivative works that are subject to Your Terms. Notwithstanding Your Terms, this License (including the redistribution requirements in Section 3.1) will continue to apply to the Work itself. + +3.3 Use Limitation. The Work and any derivative works thereof only may be used or intended for use with the web services, computing platforms or applications provided by Amazon.com, Inc. or its affiliates, including Amazon Web Services, Inc. + +3.4 Patent Claims. If you bring or threaten to bring a patent claim against any Licensor (including any claim, cross-claim or counterclaim in a lawsuit) to enforce any patents that you allege are infringed by any Work, then your rights under this License from such Licensor (including the grants in Sections 2.1 and 2.2) will terminate immediately. + +3.5 Trademarks. This License does not grant any rights to use any Licensor’s or its affiliates’ names, logos, or trademarks, except as necessary to reproduce the notices described in this License. + +3.6 Termination. If you violate any term of this License, then your rights under this License (including the grants in Sections 2.1 and 2.2) will terminate immediately. +4. Disclaimer of Warranty. + +THE WORK IS PROVIDED “AS IS” WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED, INCLUDING WARRANTIES OR CONDITIONS OF M ERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE, TITLE OR NON-INFRINGEMENT. YOU BEAR THE RISK OF UNDERTAKING ANY ACTIVITIES UNDER THIS LICENSE. SOME STATES’ CONSUMER LAWS DO NOT ALLOW EXCLUSION OF AN IMPLIED WARRANTY, SO THIS DISCLAIMER MAY NOT APPLY TO YOU. +5. Limitation of Liability. + +EXCEPT AS PROHIBITED BY APPLICABLE LAW, IN NO EVENT AND UNDER NO LEGAL THEORY, WHETHER IN TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE SHALL ANY LICENSOR BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES ARISING OUT OF OR RELATED TO THIS LICENSE, THE USE OR INABILITY TO USE THE WORK (INCLUDING BUT NOT LIMITED TO LOSS OF GOODWILL, BUSINESS INTERRUPTION, LOST PROFITS OR DATA, COMPUTER FAILURE OR MALFUNCTION, OR ANY OTHER COMM ERCIAL DAMAGES OR LOSSES), EVEN IF THE LICENSOR HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.protobuf b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.protobuf new file mode 100644 index 0000000000000..19b305b00060a --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/licenses/LICENSE.protobuf @@ -0,0 +1,32 @@ +Copyright 2008 Google Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. From 5186f5c5ba7e0b7e475bd96ce15194e4ac4289b3 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 10 Jan 2019 14:46:34 +0100 Subject: [PATCH 286/359] [hotfix][build] Append shade-plugin transformers in child modules --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b6e346a2d353d..5fd305a7514d5 100644 --- a/pom.xml +++ b/pom.xml @@ -1520,7 +1520,7 @@ under the License. org.apache.flink:force-shading - + From 7d972b100f7fa2196c9f2f19261fea535a0128bb Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 9 Jan 2019 16:20:51 +0100 Subject: [PATCH 287/359] [FLINK-11289][examples] Rework examples to account for licensing --- NOTICE-binary | 63 ++++++++++- .../flink-connector-twitter/pom.xml | 3 - flink-dist/pom.xml | 14 +++ flink-dist/src/main/assemblies/bin.xml | 24 ++++ .../pom.xml | 106 ++++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 9 ++ .../flink-examples-streaming-twitter/pom.xml | 94 ++++++++++++++++ .../flink-examples-build-helper/pom.xml | 40 +++++++ .../flink-examples-streaming/pom.xml | 74 ------------ flink-examples/pom.xml | 1 + 10 files changed, 347 insertions(+), 81 deletions(-) create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE create mode 100644 flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml create mode 100644 flink-examples/flink-examples-build-helper/pom.xml diff --git a/NOTICE-binary b/NOTICE-binary index 2619dbab859a6..9028f72aaa4f3 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -9,12 +9,8 @@ Copyright 2014-2018 The Apache Software Foundation This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.chuusai:shapeless_2.11:2.3.2 -- com.google.guava:guava:14.0.1 -- com.twitter:hbc-core:2.2.0 -- com.twitter:joauth:6.0.2 - log4j:log4j:1.2.17 - net.sf.opencsv:opencsv:2.3 -- org.apache.kafka:kafka-clients:jar:0.10.2.1 - org.scalanlp:breeze_2.11:0.13 - org.scalanlp:breeze-macros_2.11:0.13 - org.typelevel:macro-compat_2.11:1.1.1 @@ -5806,3 +5802,62 @@ This project bundles the following dependencies under the Apache Software Licens - io.dropwizard.metrics:metrics-core:3.1.5 - io.dropwizard.metrics:metrics-graphite:3.1.5 + +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for Apache Flink +// ------------------------------------------------------------------ + +Apache Flink +Copyright 2006-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +flink-examples-streaming-state-machine +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.kafka:kafka-clients:0.10.2.1 + + +flink-connector-kafka-0.10 +Copyright 2014-2019 The Apache Software Foundation + +flink-connector-kafka-0.9 +Copyright 2014-2019 The Apache Software Foundation + +flink-connector-kafka-base +Copyright 2014-2019 The Apache Software Foundation + +// ------------------------------------------------------------------ +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for Apache Flink +// ------------------------------------------------------------------ + +Apache Flink +Copyright 2006-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + + +flink-examples-streaming-twitter +Copyright 2014-2019 The Apache Software Foundation + +Apache HttpCore +Copyright 2005-2017 The Apache Software Foundation + +flink-connector-twitter +Copyright 2014-2018 The Apache Software Foundation + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.google.guava:guava:14.0.1 +- com.twitter:hbc-core:2.2.0 +- com.twitter:joauth:6.0.2 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 + +Apache HttpClient +Copyright 1999-2017 The Apache Software Foundation \ No newline at end of file diff --git a/flink-connectors/flink-connector-twitter/pom.xml b/flink-connectors/flink-connector-twitter/pom.xml index 27c7cbf6730ef..729c8c5808616 100644 --- a/flink-connectors/flink-connector-twitter/pom.xml +++ b/flink-connectors/flink-connector-twitter/pom.xml @@ -94,9 +94,6 @@ under the License. - - - diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index bd5d6bccc2f6f..deedb3027a900 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -207,6 +207,20 @@ under the License. provided + + org.apache.flink + flink-examples-streaming-state-machine_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-examples-streaming-twitter_${scala.binary.version} + ${project.version} + provided + + org.apache.flink flink-gelly-examples_${scala.binary.version} diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 8627f1a56a18a..438321c9ef4cb 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -194,6 +194,30 @@ under the License. + + + ../flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/target + examples/streaming + 0644 + + *.jar + + + original-*.jar + + + + ../flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/target + examples/streaming + 0644 + + *.jar + + + original-*.jar + + + ../flink-libraries/flink-gelly-examples/target diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml new file mode 100644 index 0000000000000..4400fb834c57a --- /dev/null +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml @@ -0,0 +1,106 @@ + + + + 4.0.0 + + + flink-examples-build-helper + org.apache.flink + 1.8-SNAPSHOT + .. + + + flink-examples-streaming-state-machine_${scala.binary.version} + flink-examples-streaming-state-machine + jar + + + + org.apache.flink + flink-examples-streaming_${scala.binary.version} + ${project.version} + + + + + StateMachineExample + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + false + false + + + org.apache.flink.streaming.examples.statemachine.StateMachineExample + + + + + org.apache.flink:flink-connector-kafka* + org.apache.flink:flink-examples-streaming* + org.apache.kafka:* + + + + + + org.apache.flink:flink-examples-streaming_* + + org/apache/flink/streaming/examples/statemachine/** + + + + org.apache.kafka:* + + LICENSE + + NOTICE + + + + + + + + + + + diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..7a2f9e33340c9 --- /dev/null +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE @@ -0,0 +1,9 @@ +flink-examples-streaming-state-machine +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.kafka:kafka-clients:0.10.2.1 diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml new file mode 100644 index 0000000000000..a602959100943 --- /dev/null +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml @@ -0,0 +1,94 @@ + + + + 4.0.0 + + + flink-examples-build-helper + org.apache.flink + 1.8-SNAPSHOT + .. + + + flink-examples-streaming-twitter_${scala.binary.version} + flink-examples-streaming-twitter + jar + + + + org.apache.flink + flink-examples-streaming_${scala.binary.version} + ${project.version} + + + + + Twitter + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + false + false + + + org.apache.flink.streaming.examples.twitter.TwitterExample + + + + + org.apache.flink:flink-connector-twitter* + org.apache.flink:flink-examples-streaming* + + + + + org.apache.flink:*flink-examples-streaming_* + + org/apache/flink/streaming/examples/twitter/** + + + + + + + + + + + diff --git a/flink-examples/flink-examples-build-helper/pom.xml b/flink-examples/flink-examples-build-helper/pom.xml new file mode 100644 index 0000000000000..344057c01a094 --- /dev/null +++ b/flink-examples/flink-examples-build-helper/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + flink-examples + org.apache.flink + 1.8-SNAPSHOT + .. + + + flink-examples-build-helper + pom + This is a utility module for building example jars to be used in flink-dist. + + + flink-examples-streaming-twitter + flink-examples-streaming-state-machine + + + diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml index 3ee304bb87dde..ecead55fad0d4 100644 --- a/flink-examples/flink-examples-streaming/pom.xml +++ b/flink-examples/flink-examples-streaming/pom.xml @@ -491,80 +491,6 @@ under the License. - - - - - org.apache.maven.plugins - maven-shade-plugin - - - - - state-machine-example - package - - shade - - - false - false - false - - - org.apache.flink.streaming.examples.statemachine.StateMachineExample - - - StateMachineExample - - - * - - org/apache/flink/streaming/examples/statemachine/** - org/apache/flink/streaming/util/serialization/** - org/apache/flink/streaming/connectors/kafka/** - org/apache/kafka/** - kafka/ - - - - - - - - fat-jar-twitter-example - package - - shade - - - false - false - false - - - org.apache.flink.streaming.examples.twitter.TwitterExample - - - Twitter - - - * - - org/apache/flink/twitter/shaded/com/google/common/** - org/apache/flink/streaming/examples/twitter/** - org/apache/flink/streaming/connectors/twitter/** - org/apache/http/** - com/twitter/** - build.properties - - - - - - - - diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 6a9b7e4379315..aa3a66f12f437 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -36,6 +36,7 @@ under the License. flink-examples-batch flink-examples-streaming flink-examples-table + flink-examples-build-helper From d5506b9191f0af0346e10561df783815e7cd0565 Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Tue, 8 Jan 2019 15:32:28 -0700 Subject: [PATCH 288/359] [FLINK-11187] [s3] Use file over stream for writes This changes the S3AccessHelper API to take a file instead of an input stream. This allows s3 client to properly reset a file instead of a file over stream for writes. This fixes an issue where the underlying s3 implementation has an intermittent failure, tries to reset the stream, fails to do so, and results in hung requests with delayed errors. --- .../utils/RefCountedBufferingFileStream.java | 7 ++----- .../s3/common/utils/RefCountedFSOutputStream.java | 8 ++++---- .../writer/RecoverableMultiPartUploadImpl.java | 9 ++++----- .../flink/fs/s3/common/writer/S3AccessHelper.java | 12 +++++------- .../utils/RefCountedBufferingFileStreamTest.java | 3 ++- .../writer/RecoverableMultiPartUploadImplTest.java | 14 +++++++------- .../S3RecoverableFsDataOutputStreamTest.java | 9 ++++----- .../flink/fs/s3hadoop/HadoopS3AccessHelper.java | 9 ++++----- 8 files changed, 32 insertions(+), 39 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStream.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStream.java index 8f3aff899f1a1..29f2590803cc8 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStream.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStream.java @@ -24,9 +24,6 @@ import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -64,8 +61,8 @@ public RefCountedBufferingFileStream( } @Override - public InputStream getInputStream() throws IOException { - return Files.newInputStream(currentTmpFile.getFile().toPath(), StandardOpenOption.READ); + public File getInputFile() { + return currentTmpFile.getFile(); } @Override diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedFSOutputStream.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedFSOutputStream.java index d4b962e146396..d51e37e8bfbf3 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedFSOutputStream.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/utils/RefCountedFSOutputStream.java @@ -21,8 +21,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.FSDataOutputStream; +import java.io.File; import java.io.IOException; -import java.io.InputStream; /** * A {@link FSDataOutputStream} with the {@link RefCounted} functionality. @@ -31,11 +31,11 @@ public abstract class RefCountedFSOutputStream extends FSDataOutputStream implements RefCounted { /** - * Gets an {@link InputStream} that allows to read the contents of the file. + * Gets the underlying {@link File} that allows to read the contents of the file. * - * @return An input stream to the contents of the file. + * @return A handle to the File object. */ - public abstract InputStream getInputStream() throws IOException; + public abstract File getInputFile(); /** * Checks if the file is closed for writes. diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java index 9d88e65f64d75..0d0998a062002 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImpl.java @@ -30,7 +30,6 @@ import java.io.File; import java.io.IOException; -import java.io.InputStream; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Deque; @@ -173,8 +172,8 @@ private String safelyUploadSmallPart(@Nullable RefCountedFSOutputStream file) th // first, upload the trailing data file. during that time, other in-progress uploads may complete. final String incompletePartObjectName = createIncompletePartObjectName(); file.retain(); - try (InputStream inputStream = file.getInputStream()) { - s3AccessHelper.putObject(incompletePartObjectName, inputStream, file.getPos()); + try { + s3AccessHelper.putObject(incompletePartObjectName, file.getInputFile()); } finally { file.release(); @@ -315,8 +314,8 @@ private static class UploadTask implements Runnable { @Override public void run() { - try (final InputStream inputStream = file.getInputStream()) { - final UploadPartResult result = s3AccessHelper.uploadPart(objectName, uploadId, partNumber, inputStream, file.getPos()); + try { + final UploadPartResult result = s3AccessHelper.uploadPart(objectName, uploadId, partNumber, file.getInputFile(), file.getPos()); future.complete(new PartETag(result.getPartNumber(), result.getETag())); file.release(); } diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java index bcdea3c00b060..593d9d36de2c1 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/writer/S3AccessHelper.java @@ -28,7 +28,6 @@ import java.io.File; import java.io.IOException; -import java.io.InputStream; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -59,25 +58,24 @@ public interface S3AccessHelper { * @param key the key this MPU is associated with. * @param uploadId the id of the MPU. * @param partNumber the number of the part being uploaded (has to be in [1 ... 10000]). - * @param file the (local) file holding the part to be uploaded. + * @param inputFile the (local) file holding the part to be uploaded. * @param length the length of the part. * @return The {@link UploadPartResult result} of the attempt to upload the part. * @throws IOException */ - UploadPartResult uploadPart(String key, String uploadId, int partNumber, InputStream file, long length) throws IOException; + UploadPartResult uploadPart(String key, String uploadId, int partNumber, File inputFile, long length) throws IOException; /** - * Uploads an object to S3. Contrary to the {@link #uploadPart(String, String, int, InputStream, long)} method, + * Uploads an object to S3. Contrary to the {@link #uploadPart(String, String, int, File, long)} method, * this object is not going to be associated to any MPU and, as such, it is not subject to the garbage collection * policies specified for your S3 bucket. * * @param key the key used to identify this part. - * @param file the (local) file holding the data to be uploaded. - * @param length the size of the data to be uploaded. + * @param inputFile the (local) file holding the data to be uploaded. * @return The {@link PutObjectResult result} of the attempt to stage the incomplete part. * @throws IOException */ - PutObjectResult putObject(String key, InputStream file, long length) throws IOException; + PutObjectResult putObject(String key, File inputFile) throws IOException; /** * Finalizes a Multi-Part Upload. diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStreamTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStreamTest.java index 5b7d1ccfb9f2d..50ea9bd64a6e1 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStreamTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/utils/RefCountedBufferingFileStreamTest.java @@ -24,6 +24,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; @@ -121,7 +122,7 @@ public void testFlush() throws IOException { Assert.assertEquals(contentToWrite.length, stream.getPos()); final byte[] contentRead = new byte[contentToWrite.length]; - stream.getInputStream().read(contentRead, 0, contentRead.length); + new FileInputStream(stream.getInputFile()).read(contentRead, 0, contentRead.length); Assert.assertTrue(Arrays.equals(contentToWrite, contentRead)); stream.release(); diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java index 673796d6c80cb..0194065633f30 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/RecoverableMultiPartUploadImplTest.java @@ -35,8 +35,8 @@ import org.junit.rules.TemporaryFolder; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -361,14 +361,14 @@ public String startMultiPartUpload(String key) throws IOException { } @Override - public UploadPartResult uploadPart(String key, String uploadId, int partNumber, InputStream file, long length) throws IOException { - final byte[] content = getFileContentBytes(file, MathUtils.checkedDownCast(length)); + public UploadPartResult uploadPart(String key, String uploadId, int partNumber, File inputFile, long length) throws IOException { + final byte[] content = getFileContentBytes(inputFile, MathUtils.checkedDownCast(length)); return storeAndGetUploadPartResult(key, partNumber, content); } @Override - public PutObjectResult putObject(String key, InputStream file, long length) throws IOException { - final byte[] content = getFileContentBytes(file, MathUtils.checkedDownCast(length)); + public PutObjectResult putObject(String key, File inputFile) throws IOException { + final byte[] content = getFileContentBytes(inputFile, MathUtils.checkedDownCast(inputFile.length())); return storeAndGetPutObjectResult(key, content); } @@ -397,9 +397,9 @@ public ObjectMetadata getObjectMetadata(String key) throws IOException { return null; } - private byte[] getFileContentBytes(InputStream file, int length) throws IOException { + private byte[] getFileContentBytes(File file, int length) throws IOException { final byte[] content = new byte[length]; - file.read(content, 0, length); + new FileInputStream(file).read(content, 0, length); return content; } diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/S3RecoverableFsDataOutputStreamTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/S3RecoverableFsDataOutputStreamTest.java index 7a32392f885e2..14ed2e294f7d2 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/S3RecoverableFsDataOutputStreamTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/writer/S3RecoverableFsDataOutputStreamTest.java @@ -39,8 +39,8 @@ import java.io.ByteArrayOutputStream; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.nio.file.FileAlreadyExistsException; @@ -314,10 +314,9 @@ private static byte[] bytesOf(String str) { private static byte[] readFileContents(RefCountedFSOutputStream file) throws IOException { final byte[] content = new byte[MathUtils.checkedDownCast(file.getPos())]; - try (InputStream inputStream = file.getInputStream()) { - int bytesRead = inputStream.read(content, 0, content.length); // TODO: 10/2/18 see if closed in download - Assert.assertEquals(file.getPos(), bytesRead); - } + File inputFile = file.getInputFile(); + long bytesRead = new FileInputStream(inputFile).read(content, 0, MathUtils.checkedDownCast(inputFile.length())); + Assert.assertEquals(file.getPos(), bytesRead); return content; } diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java index b9612adf63a66..4d37ce0ba058d 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/flink/fs/s3hadoop/HadoopS3AccessHelper.java @@ -38,7 +38,6 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -68,15 +67,15 @@ public String startMultiPartUpload(String key) throws IOException { } @Override - public UploadPartResult uploadPart(String key, String uploadId, int partNumber, InputStream inputStream, long length) throws IOException { + public UploadPartResult uploadPart(String key, String uploadId, int partNumber, File inputFile, long length) throws IOException { final UploadPartRequest uploadRequest = s3accessHelper.newUploadPartRequest( - key, uploadId, partNumber, MathUtils.checkedDownCast(length), inputStream, null, 0L); + key, uploadId, partNumber, MathUtils.checkedDownCast(length), null, inputFile, 0L); return s3accessHelper.uploadPart(uploadRequest); } @Override - public PutObjectResult putObject(String key, InputStream inputStream, long length) throws IOException { - final PutObjectRequest putRequest = s3accessHelper.createPutObjectRequest(key, inputStream, length); + public PutObjectResult putObject(String key, File inputFile) throws IOException { + final PutObjectRequest putRequest = s3accessHelper.createPutObjectRequest(key, inputFile); return s3accessHelper.putObject(putRequest); } From 769721bfbb02ea0571b850cd17aaba5d24fda4e8 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 16 Jan 2019 15:09:15 +0100 Subject: [PATCH 289/359] [FLINK-11289][examples] Fix version --- .../flink-examples-streaming-state-machine/pom.xml | 2 +- .../flink-examples-streaming-twitter/pom.xml | 2 +- flink-examples/flink-examples-build-helper/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml index 4400fb834c57a..c120db2276f6c 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples-build-helper org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT .. diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml index a602959100943..74c5b3bd1055e 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-twitter/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples-build-helper org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT .. diff --git a/flink-examples/flink-examples-build-helper/pom.xml b/flink-examples/flink-examples-build-helper/pom.xml index 344057c01a094..30c6b2397856c 100644 --- a/flink-examples/flink-examples-build-helper/pom.xml +++ b/flink-examples/flink-examples-build-helper/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples org.apache.flink - 1.8-SNAPSHOT + 1.7-SNAPSHOT .. From f2635afb94e0164987df55fc4f4838cb6f4580f3 Mon Sep 17 00:00:00 2001 From: Artsem Semianenka Date: Thu, 10 Jan 2019 17:06:00 +0100 Subject: [PATCH 290/359] [FLINK-11302][fs-connector] Correctly parse tmp dirs in FlinkS3FileSystem. --- .../s3/common/AbstractS3FileSystemFactory.java | 7 +++++-- .../fs/s3/common/S3EntropyFsFactoryTest.java | 18 ++++++++++++++++++ 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java index 6ccdeae7df835..ff575be6f55c8 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/AbstractS3FileSystemFactory.java @@ -21,11 +21,12 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystemFactory; import org.apache.flink.fs.s3.common.writer.S3AccessHelper; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,7 +139,9 @@ public FileSystem create(URI fsUri) throws IOException { } } - final String localTmpDirectory = flinkConfig.getString(CoreOptions.TMP_DIRS); + final String[] localTmpDirectories = ConfigurationUtils.parseTempDirectories(flinkConfig); + Preconditions.checkArgument(localTmpDirectories.length > 0); + final String localTmpDirectory = localTmpDirectories[0]; final long s3minPartSize = flinkConfig.getLong(PART_UPLOAD_MIN_SIZE); final int maxConcurrentUploads = flinkConfig.getInteger(MAX_CONCURRENT_UPLOADS); final S3AccessHelper s3AccessHelper = getS3AccessHelper(fs); diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java index 5b15652db62f9..943de1d889727 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/S3EntropyFsFactoryTest.java @@ -52,6 +52,24 @@ public void testEntropyInjectionConfig() throws Exception { assertEquals(7, fs.generateEntropy().length()); } + /** + * Test validates that the produced by AbstractS3FileSystemFactory object will contains + * only first path from multiple paths in config. + */ + @Test + public void testMultipleTempDirsConfig() throws Exception { + final Configuration conf = new Configuration(); + String dir1 = "/tmp/dir1"; + String dir2 = "/tmp/dir2"; + conf.setString("io.tmp.dirs", dir1 + "," + dir2); + + TestFsFactory factory = new TestFsFactory(); + factory.configure(conf); + + FlinkS3FileSystem fs = (FlinkS3FileSystem) factory.create(new URI("s3://test")); + assertEquals(fs.getLocalTmpDir(), dir1); + } + // ------------------------------------------------------------------------ private static final class TestFsFactory extends AbstractS3FileSystemFactory { From e341b3411e1706413b39e4301c97456489005044 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Wed, 16 Jan 2019 16:01:40 +0800 Subject: [PATCH 291/359] [hotfix][docs] Fix quick start project structure error in DataStream API Tutorial This closes #7502 --- docs/tutorials/datastream_api.md | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/docs/tutorials/datastream_api.md b/docs/tutorials/datastream_api.md index 4f93646c6fe45..b0964ee564ac5 100644 --- a/docs/tutorials/datastream_api.md +++ b/docs/tutorials/datastream_api.md @@ -71,9 +71,7 @@ wiki-edits/ ├── java │   └── wikiedits │   ├── BatchJob.java - │   ├── SocketTextStreamWordCount.java - │   ├── StreamingJob.java - │   └── WordCount.java + │   └── StreamingJob.java └── resources └── log4j.properties {% endhighlight %} @@ -294,7 +292,7 @@ your own machine and writing results to [Kafka](http://kafka.apache.org). ## Bonus Exercise: Running on a Cluster and Writing to Kafka Please follow our [local setup tutorial](local_setup.html) for setting up a Flink distribution -on your machine and refer to the [Kafka quickstart](https://kafka.apache.org/documentation.html#quickstart) +on your machine and refer to the [Kafka quickstart](https://kafka.apache.org/0110/documentation.html#quickstart) for setting up a Kafka installation before we proceed. As a first step, we have to add the Flink Kafka connector as a dependency so that we can @@ -303,7 +301,7 @@ use the Kafka sink. Add this to the `pom.xml` file in the dependencies section: {% highlight xml %} org.apache.flink - flink-connector-kafka-0.8_2.11 + flink-connector-kafka-0.11_2.11 ${flink.version} {% endhighlight %} @@ -320,12 +318,12 @@ result return tuple.toString(); } }) - .addSink(new FlinkKafkaProducer08<>("localhost:9092", "wiki-result", new SimpleStringSchema())); + .addSink(new FlinkKafkaProducer011<>("localhost:9092", "wiki-result", new SimpleStringSchema())); {% endhighlight %} The related classes also need to be imported: {% highlight java %} -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer08; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.common.functions.MapFunction; {% endhighlight %} @@ -355,7 +353,7 @@ We also have to create the Kafka Topic, so that our program can write to it: {% highlight bash %} $ cd my/kafka/directory -$ bin/kafka-topics.sh --create --zookeeper localhost:2181 --topic wiki-results +$ bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic wiki-results {% endhighlight %} Now we are ready to run our jar file on the local Flink cluster: From 482116806bf9c4159a505d2c013dcc298e7c8174 Mon Sep 17 00:00:00 2001 From: Igal Shilman Date: Sun, 27 Jan 2019 08:35:52 +0100 Subject: [PATCH 292/359] [FLINK-11436] [avro] Manually Java-deserialize AvroSerializer for backwards compatibility During the release of Flink 1.7, the value of serialVersionUID was uptick to 2L (was 1L before) And although the AvroSerializer (along with it's snapshot class) were migrated to the new serialization abstraction (hence free from Java serialization), there were composite serializers that were not migrated and were serialized with Java serialization. This commit manually Java-Deserializes the AvroSerializer to support backwards compatability. This closes #7580. --- .../apache/flink/util/InstantiationUtil.java | 11 +- .../formats/avro/typeutils/AvroFactory.java | 6 +- .../avro/typeutils/AvroSerializer.java | 106 +++++++++- .../AvroSerializerMigrationTest.java | 187 ++++++++++++++++++ 4 files changed, 296 insertions(+), 14 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index eab8f4ccde677..f4600a2812228 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -210,11 +210,12 @@ protected ObjectStreamClass readClassDescriptor() throws IOException, ClassNotFo final Class localClass = resolveClass(streamClassDescriptor); final String name = localClass.getName(); - if (scalaSerializerClassnames.contains(name) || scalaTypes.contains(name) || isAnonymousClass(localClass)) { + if (scalaSerializerClassnames.contains(name) || scalaTypes.contains(name) || isAnonymousClass(localClass) + || isOldAvroSerializer(name, streamClassDescriptor.getSerialVersionUID())) { final ObjectStreamClass localClassDescriptor = ObjectStreamClass.lookup(localClass); if (localClassDescriptor != null && localClassDescriptor.getSerialVersionUID() != streamClassDescriptor.getSerialVersionUID()) { - LOG.warn("Ignoring serialVersionUID mismatch for anonymous class {}; was {}, now {}.", + LOG.warn("Ignoring serialVersionUID mismatch for class {}; was {}, now {}.", streamClassDescriptor.getName(), streamClassDescriptor.getSerialVersionUID(), localClassDescriptor.getSerialVersionUID()); streamClassDescriptor = localClassDescriptor; @@ -223,6 +224,7 @@ protected ObjectStreamClass readClassDescriptor() throws IOException, ClassNotFo return streamClassDescriptor; } + } private static boolean isAnonymousClass(Class clazz) { @@ -242,6 +244,11 @@ private static boolean isAnonymousClass(Class clazz) { } } + private static boolean isOldAvroSerializer(String name, long serialVersionUID) { + // please see FLINK-11436 for details on why we need to ignore serial version UID here for the AvroSerializer + return (serialVersionUID == 1) && "org.apache.flink.formats.avro.typeutils.AvroSerializer".equals(name); + } + /** * A mapping between the full path of a deprecated serializer and its equivalent. * These mappings are hardcoded and fixed. diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java index 0ca25bfaa04e4..9a8bdcba1ca2c 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java @@ -86,9 +86,9 @@ static AvroFactory create(Class type, @Nullable Schema currentSchema, return fromReflective(type, cl, Optional.ofNullable(previousSchema)); } - static AvroFactory createFromTypeAndSchemaString(Class type, @Nullable String schemaString) { - Schema schema = (schemaString != null) ? new Schema.Parser().parse(schemaString) : null; - return create(type, schema, null); + @Nullable + static Schema parseSchemaString(@Nullable String schemaString) { + return (schemaString == null) ? null : new Schema.Parser().parse(schemaString); } @SuppressWarnings("OptionalUsedAsFieldOrParameterType") diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java index 2be660ad27be0..da51117984d95 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java @@ -32,12 +32,15 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumReader; import org.apache.avro.io.DatumWriter; -import org.apache.avro.reflect.Nullable; import org.apache.avro.specific.SpecificRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + import java.io.IOException; +import java.io.ObjectInputStream; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -75,11 +78,9 @@ public class AvroSerializer extends TypeSerializer { // -------- configuration fields, serializable ----------- - /** The class of the type that is serialized by this serializer. - */ - private final Class type; - private final SerializableAvroSchema schema; - private final SerializableAvroSchema previousSchema; + @Nonnull private Class type; + @Nonnull private SerializableAvroSchema schema; + @Nonnull private SerializableAvroSchema previousSchema; // -------- runtime fields, non-serializable, lazily initialized ----------- @@ -127,10 +128,10 @@ public AvroSerializer(Class type, Schema schema) { * Creates a new AvroSerializer for the type indicated by the given class. */ @Internal - AvroSerializer(Class type, @Nullable SerializableAvroSchema newSchema, @Nullable SerializableAvroSchema previousSchema) { + AvroSerializer(Class type, SerializableAvroSchema newSchema, SerializableAvroSchema previousSchema) { this.type = checkNotNull(type); - this.schema = newSchema; - this.previousSchema = previousSchema; + this.schema = checkNotNull(newSchema); + this.previousSchema = checkNotNull(previousSchema); } /** @@ -144,6 +145,7 @@ public AvroSerializer(Class type, Class typeToInstantiate) { // ------------------------------------------------------------------------ + @Nonnull public Class getType() { return type; } @@ -381,4 +383,90 @@ public AvroSchemaSerializerConfigSnapshot() { } } + + // -------- backwards compatibility with 1.5, 1.6 ----------- + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + /* + Please see FLINK-11436 for details on why manual deserialization is required. + + During the release of Flink 1.7, the value of serialVersionUID was uptick to 2L (was 1L before) + And although the AvroSerializer (along with it's snapshot class) were migrated to the new serialization + abstraction (hence free from Java serialization), there were composite serializers that were not migrated + and were serialized with Java serialization. In case that one of the nested serializers were Avro we would + bump into deserialization exception due to a wrong serialVersionUID. Unfortunately it is not possible to revert + the serialVersionUID back to 1L, because users might have snapshots with 2L present already. + To overcome this we first need to make sure that the AvroSerializer is being Java deserialized with + FailureTolerantObjectInputStream, and then we determine the serialized layout by looking at the fields. + + From: https://docs.oracle.com/javase/8/docs/platform/serialization/spec/class.html#a5421 + ------------------------------------------------------------------------------------------------------------- + The descriptors for primitive typed fields are written first + sorted by field name followed by descriptors for the object typed fields sorted by field name. + The names are sorted using String.compareTo. + ------------------------------------------------------------------------------------------------------------- + + pre 1.6 field order: [type] + pre 1.7 field order: [schemaString, type] + post 1.7 field order: [previousSchema, schema, type] + + We would use the first field to distinguish between the three different layouts. + To complicate things even further in pre 1.7, the field @schemaString could be + null or a string, but, in post 1.7, the field @previousSchema was never set to null, therefore + we can use the first field to determine the version. + + this logic should stay here as long as we support Flink 1.6 (along with Java serialized + TypeSerializers) + */ + final Object firstField = in.readObject(); + + if (firstField == null) { + // first field can only be NULL in 1.6 (schemaString) + read16Layout(null, in); + } + else if (firstField instanceof String) { + // first field is a String only in 1.6 (schemaString) + read16Layout((String) firstField, in); + } + else if (firstField instanceof Class) { + // first field is a Class only in 1.5 (type) + @SuppressWarnings("unchecked") Class type = (Class) firstField; + read15Layout(type); + } + else if (firstField instanceof SerializableAvroSchema) { + readCurrentLayout((SerializableAvroSchema) firstField, in); + } + else { + throw new IllegalStateException("Failed to Java-Deserialize an AvroSerializer instance. " + + "Was expecting a first field to be either a String or SerializableAvroSchema, but got: " + + "" + firstField.getClass()); + } + } + + private void read15Layout(Class type) { + this.previousSchema = new SerializableAvroSchema(); + this.schema = new SerializableAvroSchema(); + this.type = type; + } + + @SuppressWarnings("unchecked") + private void read16Layout(@Nullable String schemaString, ObjectInputStream in) + throws IOException, ClassNotFoundException { + + Schema schema = AvroFactory.parseSchemaString(schemaString); + Class type = (Class) in.readObject(); + + this.previousSchema = new SerializableAvroSchema(); + this.schema = new SerializableAvroSchema(schema); + this.type = type; + } + + @SuppressWarnings("unchecked") + private void readCurrentLayout(SerializableAvroSchema previousSchema, ObjectInputStream in) + throws IOException, ClassNotFoundException { + + this.previousSchema = previousSchema; + this.schema = (SerializableAvroSchema) in.readObject(); + this.type = (Class) in.readObject(); + } } diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerMigrationTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerMigrationTest.java index a5f2a8e128f01..8b35a929f5a82 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerMigrationTest.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerMigrationTest.java @@ -18,15 +18,33 @@ package org.apache.flink.formats.avro.typeutils; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshotMigrationTestBase; +import org.apache.flink.core.memory.DataInputDeserializer; import org.apache.flink.formats.avro.generated.Address; +import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.avro.generic.GenericRecord; +import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; import java.util.Arrays; +import java.util.Base64; import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static junit.framework.TestCase.assertSame; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; /** * Tests migrations for {@link AvroSerializerSnapshot}. @@ -62,4 +80,173 @@ public static Collection testSpecifications() { ); } + // --------------------------------------------------------------------------------------------------------------- + // The following batch of tests are making sure that AvroSerializer class is able to be Java-Deserialized. + // see [FLINK-11436] for more information. + + // Once we drop support for versions that carried snapshots with Java-Deserialized serializers we can drop this + // batch of tests. + // --------------------------------------------------------------------------------------------------------------- + + @Test + public void javaDeserializeFromFlink_1_5_ReflectiveRecord() throws IOException { + final String avroSerializerBase64 = "AAAAAQAAAQis7QAFc3IANm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJv\n" + + "U2VyaWFsaXplcgAAAAAAAAABAgABTAAEdHlwZXQAEUxqYXZhL2xhbmcvQ2xhc3M7eHIANG9yZy5hcGFj\n" + + "aGUuZmxpbmsuYXBpLmNvbW1vbi50eXBldXRpbHMuVHlwZVNlcmlhbGl6ZXIAAAAAAAAAAQIAAHhwdnIA\n" + + "Tm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJvU2VyaWFsaXplck1pZ3Jh\n" + + "dGlvblRlc3QkU2ltcGxlUG9qbwAAAAAAAAAAAAAAeHA="; + + TypeSerializer serializer = javaDeserialize(avroSerializerBase64); + assertThat(serializer, instanceOf(AvroSerializer.class)); + + AvroSerializer avroSerializer = (AvroSerializer) javaDeserialize(avroSerializerBase64); + assertSame(avroSerializer.getType(), SimplePojo.class); + assertThat(avroSerializer.getAvroSchema(), notNullValue()); + } + + @Test + public void javaDeserializeFromFlink_1_5_SpecificRecord() throws IOException { + final String avroSerializerBase64 = "AAAAAQAAASOs7QAFc3IANm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJv\n" + + "U2VyaWFsaXplcgAAAAAAAAABAgABTAAEdHlwZXQAEUxqYXZhL2xhbmcvQ2xhc3M7eHIANG9yZy5hcGFj\n" + + "aGUuZmxpbmsuYXBpLmNvbW1vbi50eXBldXRpbHMuVHlwZVNlcmlhbGl6ZXIAAAAAAAAAAQIAAHhwdnIA\n" + + "L29yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLmdlbmVyYXRlZC5BZGRyZXNz7Paj+KjgQ2oMAAB4\n" + + "cgArb3JnLmFwYWNoZS5hdnJvLnNwZWNpZmljLlNwZWNpZmljUmVjb3JkQmFzZQKi+azGtzQdDAAAeHA="; + + TypeSerializer serializer = javaDeserialize(avroSerializerBase64); + assertThat(serializer, instanceOf(AvroSerializer.class)); + + AvroSerializer avroSerializer = (AvroSerializer) javaDeserialize(avroSerializerBase64); + assertSame(avroSerializer.getType(), Address.class); + assertThat(avroSerializer.getAvroSchema(), is(Address.SCHEMA$)); + } + + @Test + public void javaDeserializeFromFlink_1_6() throws IOException { + final String avroSerializer = "AAAAAQAAAUis7QAFc3IANm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJv\n" + + "U2VyaWFsaXplcgAAAAAAAAABAgACTAAMc2NoZW1hU3RyaW5ndAASTGphdmEvbGFuZy9TdHJpbmc7TAAE\n" + + "dHlwZXQAEUxqYXZhL2xhbmcvQ2xhc3M7eHIANG9yZy5hcGFjaGUuZmxpbmsuYXBpLmNvbW1vbi50eXBl\n" + + "dXRpbHMuVHlwZVNlcmlhbGl6ZXIAAAAAAAAAAQIAAHhwcHZyAC9vcmcuYXBhY2hlLmZsaW5rLmZvcm1h\n" + + "dHMuYXZyby5nZW5lcmF0ZWQuQWRkcmVzc+z2o/io4ENqDAAAeHIAK29yZy5hcGFjaGUuYXZyby5zcGVj\n" + + "aWZpYy5TcGVjaWZpY1JlY29yZEJhc2UCovmsxrc0HQwAAHhw"; + + TypeSerializer avro = javaDeserialize(avroSerializer); + + assertThat(avro, instanceOf(AvroSerializer.class)); + } + + @Test + public void javaDeserializeFromFlink_1_6_GenericRecord() throws IOException { + String avroSerializerBase64 = "AAAAAQAAAges7QAFc3IANm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJv\n" + + "U2VyaWFsaXplcgAAAAAAAAABAgACTAAMc2NoZW1hU3RyaW5ndAASTGphdmEvbGFuZy9TdHJpbmc7TAAE\n" + + "dHlwZXQAEUxqYXZhL2xhbmcvQ2xhc3M7eHIANG9yZy5hcGFjaGUuZmxpbmsuYXBpLmNvbW1vbi50eXBl\n" + + "dXRpbHMuVHlwZVNlcmlhbGl6ZXIAAAAAAAAAAQIAAHhwdAEBeyJ0eXBlIjoicmVjb3JkIiwibmFtZSI6\n" + + "IkFkZHJlc3MiLCJuYW1lc3BhY2UiOiJvcmcuYXBhY2hlLmZsaW5rLmZvcm1hdHMuYXZyby5nZW5lcmF0\n" + + "ZWQiLCJmaWVsZHMiOlt7Im5hbWUiOiJudW0iLCJ0eXBlIjoiaW50In0seyJuYW1lIjoic3RyZWV0Iiwi\n" + + "dHlwZSI6InN0cmluZyJ9LHsibmFtZSI6ImNpdHkiLCJ0eXBlIjoic3RyaW5nIn0seyJuYW1lIjoic3Rh\n" + + "dGUiLCJ0eXBlIjoic3RyaW5nIn0seyJuYW1lIjoiemlwIiwidHlwZSI6InN0cmluZyJ9XX12cgAlb3Jn\n" + + "LmFwYWNoZS5hdnJvLmdlbmVyaWMuR2VuZXJpY1JlY29yZAAAAAAAAAAAAAAAeHA="; + + TypeSerializer serializer = javaDeserialize(avroSerializerBase64); + + AvroSerializer avroSerializer = (AvroSerializer) serializer; + assertSame(avroSerializer.getType(), GenericRecord.class); + assertThat(avroSerializer.getAvroSchema(), notNullValue()); + } + + @Test + public void javaDeserializeFromFlink_1_7() throws IOException { + String avroSerializerBase64 = "AAAAAQAAAeKs7QAFc3IANm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJv\n" + + "U2VyaWFsaXplcgAAAAAAAAACAgADTAAOcHJldmlvdXNTY2hlbWF0AEBMb3JnL2FwYWNoZS9mbGluay9m\n" + + "b3JtYXRzL2F2cm8vdHlwZXV0aWxzL1NlcmlhbGl6YWJsZUF2cm9TY2hlbWE7TAAGc2NoZW1hcQB+AAFM\n" + + "AAR0eXBldAARTGphdmEvbGFuZy9DbGFzczt4cgA0b3JnLmFwYWNoZS5mbGluay5hcGkuY29tbW9uLnR5\n" + + "cGV1dGlscy5UeXBlU2VyaWFsaXplcgAAAAAAAAABAgAAeHBzcgA+b3JnLmFwYWNoZS5mbGluay5mb3Jt\n" + + "YXRzLmF2cm8udHlwZXV0aWxzLlNlcmlhbGl6YWJsZUF2cm9TY2hlbWEAAAAAAAAAAQMAAHhwdwEAeHNx\n" + + "AH4ABXcBAHh2cgAvb3JnLmFwYWNoZS5mbGluay5mb3JtYXRzLmF2cm8uZ2VuZXJhdGVkLkFkZHJlc3Ps\n" + + "9qP4qOBDagwAAHhyACtvcmcuYXBhY2hlLmF2cm8uc3BlY2lmaWMuU3BlY2lmaWNSZWNvcmRCYXNlAqL5\n" + + "rMa3NB0MAAB4cA=="; + + AvroSerializer avroSerializer = (AvroSerializer) javaDeserialize(avroSerializerBase64); + assertSame(avroSerializer.getType(), Address.class); + assertThat(avroSerializer.getAvroSchema(), is(Address.SCHEMA$)); + } + + @Test + public void javaDeserializeFromFlink_1_7_afterInitialization() throws IOException { + String avroSerializerBase64 = "AAAAAQAAAeKs7QAFc3IANm9yZy5hcGFjaGUuZmxpbmsuZm9ybWF0cy5hdnJvLnR5cGV1dGlscy5BdnJv\n" + + "U2VyaWFsaXplcgAAAAAAAAACAgADTAAOcHJldmlvdXNTY2hlbWF0AEBMb3JnL2FwYWNoZS9mbGluay9m\n" + + "b3JtYXRzL2F2cm8vdHlwZXV0aWxzL1NlcmlhbGl6YWJsZUF2cm9TY2hlbWE7TAAGc2NoZW1hcQB+AAFM\n" + + "AAR0eXBldAARTGphdmEvbGFuZy9DbGFzczt4cgA0b3JnLmFwYWNoZS5mbGluay5hcGkuY29tbW9uLnR5\n" + + "cGV1dGlscy5UeXBlU2VyaWFsaXplcgAAAAAAAAABAgAAeHBzcgA+b3JnLmFwYWNoZS5mbGluay5mb3Jt\n" + + "YXRzLmF2cm8udHlwZXV0aWxzLlNlcmlhbGl6YWJsZUF2cm9TY2hlbWEAAAAAAAAAAQMAAHhwdwEAeHNx\n" + + "AH4ABXcBAHh2cgAvb3JnLmFwYWNoZS5mbGluay5mb3JtYXRzLmF2cm8uZ2VuZXJhdGVkLkFkZHJlc3Ps\n" + + "9qP4qOBDagwAAHhyACtvcmcuYXBhY2hlLmF2cm8uc3BlY2lmaWMuU3BlY2lmaWNSZWNvcmRCYXNlAqL5\n" + + "rMa3NB0MAAB4cA=="; + + AvroSerializer avroSerializer = (AvroSerializer) javaDeserialize(avroSerializerBase64); + assertSame(avroSerializer.getType(), Address.class); + assertThat(avroSerializer.getAvroSchema(), is(Address.SCHEMA$)); + } + + @Test + public void compositeSerializerFromFlink_1_6_WithNestedAvroSerializer() throws IOException { + String streamElementSerializerBase64 = "AAAAAQAAAq2s7QAFc3IAR29yZy5hcGFjaGUuZmxpbmsuc3RyZWFtaW5nLnJ1bnRpbWUuc3RyZWFtcmVj\n" + + "b3JkLlN0cmVhbUVsZW1lbnRTZXJpYWxpemVyAAAAAAAAAAECAAFMAA50eXBlU2VyaWFsaXplcnQANkxv\n" + + "cmcvYXBhY2hlL2ZsaW5rL2FwaS9jb21tb24vdHlwZXV0aWxzL1R5cGVTZXJpYWxpemVyO3hyADRvcmcu\n" + + "YXBhY2hlLmZsaW5rLmFwaS5jb21tb24udHlwZXV0aWxzLlR5cGVTZXJpYWxpemVyAAAAAAAAAAECAAB4\n" + + "cHNyADZvcmcuYXBhY2hlLmZsaW5rLmZvcm1hdHMuYXZyby50eXBldXRpbHMuQXZyb1NlcmlhbGl6ZXIA\n" + + "AAAAAAAAAQIAAkwADHNjaGVtYVN0cmluZ3QAEkxqYXZhL2xhbmcvU3RyaW5nO0wABHR5cGV0ABFMamF2\n" + + "YS9sYW5nL0NsYXNzO3hxAH4AAnQBAXsidHlwZSI6InJlY29yZCIsIm5hbWUiOiJBZGRyZXNzIiwibmFt\n" + + "ZXNwYWNlIjoib3JnLmFwYWNoZS5mbGluay5mb3JtYXRzLmF2cm8uZ2VuZXJhdGVkIiwiZmllbGRzIjpb\n" + + "eyJuYW1lIjoibnVtIiwidHlwZSI6ImludCJ9LHsibmFtZSI6InN0cmVldCIsInR5cGUiOiJzdHJpbmci\n" + + "fSx7Im5hbWUiOiJjaXR5IiwidHlwZSI6InN0cmluZyJ9LHsibmFtZSI6InN0YXRlIiwidHlwZSI6InN0\n" + + "cmluZyJ9LHsibmFtZSI6InppcCIsInR5cGUiOiJzdHJpbmcifV19dnIAJW9yZy5hcGFjaGUuYXZyby5n\n" + + "ZW5lcmljLkdlbmVyaWNSZWNvcmQAAAAAAAAAAAAAAHhw"; + + StreamElementSerializer ser = (StreamElementSerializer) javaDeserialize(streamElementSerializerBase64); + TypeSerializer containedTypeSerializer = ser.getContainedTypeSerializer(); + + assertThat(containedTypeSerializer, instanceOf(AvroSerializer.class)); + + AvroSerializer avroSerializer = (AvroSerializer) containedTypeSerializer; + assertSame(avroSerializer.getType(), GenericRecord.class); + assertThat(avroSerializer.getAvroSchema(), is(Address.SCHEMA$)); + } + + @Test + public void makeSureThatFieldsWereNotChanged() { + // This test should be removed once we completely migrate all the composite serializers. + + List serializedFieldNames = Arrays.stream(AvroSerializer.class.getDeclaredFields()) + .filter(field -> !Modifier.isTransient(field.getModifiers())) + .filter(field -> !Modifier.isStatic(field.getModifiers())) + .map(Field::getName) + .sorted() + .collect(Collectors.toList()); + + assertThat(serializedFieldNames, is(asList("previousSchema", "schema", "type"))); + } + + @SuppressWarnings("deprecation") + private static TypeSerializer javaDeserialize(String base64) throws IOException { + byte[] bytes = Base64.getMimeDecoder().decode(base64); + DataInputDeserializer in = new DataInputDeserializer(bytes); + return TypeSerializerSerializationUtil.tryReadSerializer(in, Thread.currentThread().getContextClassLoader()); + } + + /** + * A simple pojo used in these tests. + */ + public static class SimplePojo { + private String foo; + + @SuppressWarnings("unused") + public String getFoo() { + return foo; + } + + @SuppressWarnings("unused") + public void setFoo(String foo) { + this.foo = foo; + } + } } From d241a419ee5be486f1b563ac8d19b48897b75bbe Mon Sep 17 00:00:00 2001 From: eaglewatcherwb Date: Tue, 8 Jan 2019 17:36:03 +0800 Subject: [PATCH 293/359] [FLINK-11071][core] add support for dynamic proxy classes resolution in job defination This closes #7436 --- .../apache/flink/util/InstantiationUtil.java | 35 +++++++++++++++ .../flink/util/InstantiationUtilTest.java | 44 +++++++++++++++++++ 2 files changed, 79 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index f4600a2812228..644289133b283 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -43,6 +43,7 @@ import java.io.Serializable; import java.lang.reflect.Constructor; import java.lang.reflect.Modifier; +import java.lang.reflect.Proxy; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -91,6 +92,40 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas return super.resolveClass(desc); } + @Override + protected Class resolveProxyClass(String[] interfaces) throws IOException, ClassNotFoundException { + if (classLoader != null) { + ClassLoader nonPublicLoader = null; + boolean hasNonPublicInterface = false; + + // define proxy in class loader of non-public interface(s), if any + Class[] classObjs = new Class[interfaces.length]; + for (int i = 0; i < interfaces.length; i++) { + Class cl = Class.forName(interfaces[i], false, classLoader); + if ((cl.getModifiers() & Modifier.PUBLIC) == 0) { + if (hasNonPublicInterface) { + if (nonPublicLoader != cl.getClassLoader()) { + throw new IllegalAccessError( + "conflicting non-public interface class loaders"); + } + } else { + nonPublicLoader = cl.getClassLoader(); + hasNonPublicInterface = true; + } + } + classObjs[i] = cl; + } + try { + return Proxy.getProxyClass( + hasNonPublicInterface ? nonPublicLoader : classLoader, classObjs); + } catch (IllegalArgumentException e) { + throw new ClassNotFoundException(null, e); + } + } + + return super.resolveProxyClass(interfaces); + } + // ------------------------------------------------ private static final HashMap> primitiveClasses = new HashMap<>(9); diff --git a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java index 0e8639109c754..3de393e7d8997 100644 --- a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java @@ -23,15 +23,22 @@ 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.testutils.ClassLoaderUtils; import org.apache.flink.types.DoubleValue; import org.apache.flink.types.StringValue; import org.apache.flink.types.Value; +import org.junit.ClassRule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.net.URLClassLoader; import java.util.Objects; import java.util.Random; @@ -46,6 +53,32 @@ */ public class InstantiationUtilTest extends TestLogger { + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void testResolveProxyClass() throws Exception { + + UserDefineFunctionImpl method = new UserDefineFunctionImpl(); + final String className = "UserDefineFunctionImpl"; + final URLClassLoader userClassLoader = ClassLoaderUtils.compileAndLoadJava( + temporaryFolder.newFolder(), + className + ".java", + "import java.io.Serializable;\n" + + "interface UserDefineFunction { void test();}\n" + + "public class " + className + " implements UserDefineFunction, Serializable {public void test() {} }"); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + InstantiationUtil.serializeObject(baos, method); + + InstantiationUtil.ClassLoaderObjectInputStream cloi = new InstantiationUtil.ClassLoaderObjectInputStream( + new ByteArrayInputStream(baos.toByteArray()), userClassLoader); + cloi.resolveProxyClass(new String[]{"UserDefineFunction"}); + + userClassLoader.close(); + } + @Test public void testInstantiationOfStringValue() { StringValue stringValue = InstantiationUtil.instantiate( @@ -153,6 +186,17 @@ public void testCopyWritable() throws Exception { // -------------------------------------------------------------------------------------------- + interface UserDefineFunction extends Serializable { + void test(); + } + + private static class UserDefineFunctionImpl implements UserDefineFunction { + @Override + public void test() { + + } + } + private class TestClass {} private static class TestException extends IOException { From 885869edb1ba46051af89450370569d62965ca3a Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Sun, 27 Jan 2019 19:13:41 +0100 Subject: [PATCH 294/359] [FLINK-11071][core] Improved proxy class serialization test This closes #7436 --- .../flink/testutils/ClassLoaderUtils.java | 54 +++++++++++++++- .../flink/util/InstantiationUtilTest.java | 62 ++++++++++--------- 2 files changed, 85 insertions(+), 31 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java b/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java index 0688c1df15638..b10f2e175b83f 100644 --- a/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java +++ b/flink-core/src/test/java/org/apache/flink/testutils/ClassLoaderUtils.java @@ -24,8 +24,11 @@ import java.io.File; import java.io.FileWriter; import java.io.IOException; +import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; +import java.util.HashMap; +import java.util.Map; /** * Utilities to create class loaders. @@ -33,15 +36,23 @@ public class ClassLoaderUtils { public static URLClassLoader compileAndLoadJava(File root, String filename, String source) throws IOException { - File file = writeSourceFile(root, filename, source); - - compileClass(file); + return withRoot(root) + .addClass(filename.replaceAll("\\.java", ""), source) + .build(); + } + private static URLClassLoader createClassLoader(File root) throws MalformedURLException { return new URLClassLoader( new URL[]{root.toURI().toURL()}, Thread.currentThread().getContextClassLoader()); } + private static void writeAndCompile(File root, String filename, String source) throws IOException { + File file = writeSourceFile(root, filename, source); + + compileClass(file); + } + private static File writeSourceFile(File root, String filename, String source) throws IOException { File file = new File(root, filename); FileWriter fileWriter = new FileWriter(file); @@ -52,8 +63,45 @@ private static File writeSourceFile(File root, String filename, String source) t return file; } + public static ClassLoaderBuilder withRoot(File root) { + return new ClassLoaderBuilder(root); + } + private static int compileClass(File sourceFile) { JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); return compiler.run(null, null, null, "-proc:none", sourceFile.getPath()); } + + public static class ClassLoaderBuilder { + + private final File root; + private final Map classes; + + private ClassLoaderBuilder(File root) { + this.root = root; + this.classes = new HashMap<>(); + } + + public ClassLoaderBuilder addClass(String className, String source) { + String oldValue = classes.putIfAbsent(className, source); + + if (oldValue != null) { + throw new RuntimeException(String.format("Class with name %s already registered.", className)); + } + + return this; + } + + public URLClassLoader build() throws IOException { + for (Map.Entry classInfo : classes.entrySet()) { + writeAndCompile(root, createFileName(classInfo.getKey()), classInfo.getValue()); + } + + return createClassLoader(root); + } + + private String createFileName(String className) { + return className + ".java"; + } + } } diff --git a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java index 3de393e7d8997..ccbb5d7d9ab4f 100644 --- a/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/InstantiationUtilTest.java @@ -32,12 +32,11 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; -import java.io.Serializable; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Proxy; import java.net.URLClassLoader; import java.util.Objects; import java.util.Random; @@ -56,27 +55,45 @@ public class InstantiationUtilTest extends TestLogger { @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static final String PROXY_DEFINITION_FORMAT = + "import java.lang.reflect.InvocationHandler;" + + "import java.lang.reflect.Method;" + + "import java.io.Serializable;" + + "public class %s implements InvocationHandler, Serializable {\n" + + "\n" + + " @Override\n" + + " public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {\n" + + " return null;\n" + + " }\n" + + "}"; + @Test public void testResolveProxyClass() throws Exception { + final String interfaceName = "UserDefinedInterface"; + final String proxyName = "UserProxy"; - UserDefineFunctionImpl method = new UserDefineFunctionImpl(); - final String className = "UserDefineFunctionImpl"; - final URLClassLoader userClassLoader = ClassLoaderUtils.compileAndLoadJava( - temporaryFolder.newFolder(), - className + ".java", - "import java.io.Serializable;\n" - + "interface UserDefineFunction { void test();}\n" - + "public class " + className + " implements UserDefineFunction, Serializable {public void test() {} }"); + try (URLClassLoader userClassLoader = createClassLoader(interfaceName, proxyName)) { + Class userInterface = Class.forName(interfaceName, false, userClassLoader); + InvocationHandler userProxy = (InvocationHandler) Class.forName(proxyName, false, userClassLoader) + .newInstance(); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); + Object proxy = Proxy.newProxyInstance(userClassLoader, new Class[]{userInterface}, userProxy); - InstantiationUtil.serializeObject(baos, method); + byte[] serializeObject = InstantiationUtil.serializeObject(proxy); + Object deserializedProxy = InstantiationUtil.deserializeObject(serializeObject, userClassLoader); + assertNotNull(deserializedProxy); + } + } - InstantiationUtil.ClassLoaderObjectInputStream cloi = new InstantiationUtil.ClassLoaderObjectInputStream( - new ByteArrayInputStream(baos.toByteArray()), userClassLoader); - cloi.resolveProxyClass(new String[]{"UserDefineFunction"}); + private URLClassLoader createClassLoader(String interfaceName, String proxyName) throws IOException { + return ClassLoaderUtils.withRoot(temporaryFolder.newFolder()) + .addClass(interfaceName, String.format("interface %s { void test();}", interfaceName)) + .addClass(proxyName, createProxyDefinition(proxyName)) + .build(); + } - userClassLoader.close(); + private String createProxyDefinition(String proxyName) { + return String.format(PROXY_DEFINITION_FORMAT, proxyName); } @Test @@ -186,17 +203,6 @@ public void testCopyWritable() throws Exception { // -------------------------------------------------------------------------------------------- - interface UserDefineFunction extends Serializable { - void test(); - } - - private static class UserDefineFunctionImpl implements UserDefineFunction { - @Override - public void test() { - - } - } - private class TestClass {} private static class TestException extends IOException { From e79bd7ac4b219a71a708084a63470d7f9309531e Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 30 Jan 2019 11:42:22 +0100 Subject: [PATCH 295/359] [FLINK-10910][e2e] Hardened Kubernetes e2e test. Added check if minikube is running. If it is not we try to start it couple of times. If we do not succeed we fail with a descriptive message. --- .../test_kubernetes_embedded_job.sh | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh index 4529fd904a687..96320fd53c50c 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh @@ -22,6 +22,8 @@ source "$(dirname "$0")"/common.sh DOCKER_MODULE_DIR=${END_TO_END_DIR}/../flink-container/docker KUBERNETES_MODULE_DIR=${END_TO_END_DIR}/../flink-container/kubernetes CONTAINER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts +MINIKUBE_START_RETRIES=3 +MINIKUBE_START_BACKOFF=5 export FLINK_JOB=org.apache.flink.examples.java.wordcount.WordCount export FLINK_IMAGE_NAME=test_kubernetes_embedded_job @@ -37,10 +39,29 @@ function cleanup { rm -rf ${OUTPUT_VOLUME} } +function check_kubernetes_status { + local status=`minikube status` + echo ${status} | grep -q "minikube: Running cluster: Running kubectl: Correctly Configured" + return $? +} + +function start_kubernetes_if_not_running { + if ! check_kubernetes_status; then + minikube start + fi + + return $(check_kubernetes_status) +} + trap cleanup EXIT mkdir -p $OUTPUT_VOLUME +if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} start_kubernetes_if_not_running; then + echo "Minikube not running. Could not start minikube. Aborting..." + exit 1 +fi + eval $(minikube docker-env) cd "$DOCKER_MODULE_DIR" ./build.sh --from-local-dist --job-jar ${FLINK_DIR}/examples/batch/WordCount.jar --image-name ${FLINK_IMAGE_NAME} From c193cf04a45674c95693a7b125a01ed2303a4876 Mon Sep 17 00:00:00 2001 From: "junsheng.wu" Date: Wed, 30 Jan 2019 14:34:45 +0100 Subject: [PATCH 296/359] [FLINK-11469][docs] Update documentation for `Tuning Checkpoints and Large State` This closes #7603. (cherry picked from commit 3abb3de79e36b04cf5a5012fc1062f4418b3597a) --- docs/ops/state/large_state_tuning.md | 22 +++++++++------------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md index 4bf890d622aa3..47154af58a4fa 100644 --- a/docs/ops/state/large_state_tuning.md +++ b/docs/ops/state/large_state_tuning.md @@ -161,21 +161,17 @@ RocksDBStateBackend.setOptions(new MyOptions()); public class MyOptions implements OptionsFactory { @Override - public DBOptions createDBOptions() { - return new DBOptions() - .setIncreaseParallelism(4) - .setUseFsync(false) - .setDisableDataSync(true); + public DBOptions createDBOptions(DBOptions currentOptions) { + return currentOptions.setIncreaseParallelism(4) + .setUseFsync(false); } - + @Override - public ColumnFamilyOptions createColumnOptions() { - - return new ColumnFamilyOptions() - .setTableFormatConfig( - new BlockBasedTableConfig() - .setBlockCacheSize(256 * 1024 * 1024) // 256 MB - .setBlockSize(128 * 1024)); // 128 KB + public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions currentOptions) { + return currentOptions.setTableFormatConfig( + new BlockBasedTableConfig() + .setBlockCacheSize(256 * 1024 * 1024) // 256 MB + .setBlockSize(128 * 1024)); // 128 KB } } {% endhighlight %} From af83b991f59bf27b01562e0b5a1e1ce2962a28b0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 4 Nov 2018 18:04:32 -0800 Subject: [PATCH 297/359] [FLINK-10774] Rework lifecycle management of partitionDiscoverer in FlinkKafkaConsumerBase --- .../kafka/FlinkKafkaConsumerBase.java | 173 +++++---- .../kafka/FlinkKafkaConsumerBaseTest.java | 334 +++++++++++++++++- .../util/MockStreamingRuntimeContext.java | 12 + 3 files changed, 424 insertions(+), 95 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index cfb5b6d510d0f..90f182a24c937 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -50,6 +50,7 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.SerializedValue; import org.apache.commons.collections.map.LinkedMap; @@ -469,9 +470,7 @@ public void open(Configuration configuration) throws Exception { this.partitionDiscoverer.open(); subscribedPartitionsToStartOffsets = new HashMap<>(); - - List allPartitions = partitionDiscoverer.discoverPartitions(); - + final List allPartitions = partitionDiscoverer.discoverPartitions(); if (restoredState != null) { for (KafkaTopicPartition partition : allPartitions) { if (!restoredState.containsKey(partition)) { @@ -485,7 +484,7 @@ public void open(Configuration configuration) throws Exception { // restored partitions that should not be subscribed by this subtask if (KafkaTopicPartitionAssigner.assign( restoredStateEntry.getKey(), getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()){ + == getRuntimeContext().getIndexOfThisSubtask()){ subscribedPartitionsToStartOffsets.put(restoredStateEntry.getKey(), restoredStateEntry.getValue()); } } else { @@ -533,16 +532,16 @@ public void open(Configuration configuration) throws Exception { } for (Map.Entry partitionToOffset - : fetchOffsetsWithTimestamp(allPartitions, startupOffsetsTimestamp).entrySet()) { + : fetchOffsetsWithTimestamp(allPartitions, startupOffsetsTimestamp).entrySet()) { subscribedPartitionsToStartOffsets.put( partitionToOffset.getKey(), (partitionToOffset.getValue() == null) - // if an offset cannot be retrieved for a partition with the given timestamp, - // we default to using the latest offset for the partition - ? KafkaTopicPartitionStateSentinel.LATEST_OFFSET - // since the specified offsets represent the next record to read, we subtract - // it by one so that the initial state of the consumer will be correct - : partitionToOffset.getValue() - 1); + // if an offset cannot be retrieved for a partition with the given timestamp, + // we default to using the latest offset for the partition + ? KafkaTopicPartitionStateSentinel.LATEST_OFFSET + // since the specified offsets represent the next record to read, we subtract + // it by one so that the initial state of the consumer will be correct + : partitionToOffset.getValue() - 1); } break; @@ -595,7 +594,6 @@ public void open(Configuration configuration) throws Exception { partitionsDefaultedToGroupOffsets); } break; - default: case GROUP_OFFSETS: LOG.info("Consumer subtask {} will start reading the following {} partitions from the committed group offsets in Kafka: {}", getRuntimeContext().getIndexOfThisSubtask(), @@ -663,80 +661,87 @@ public void onException(Throwable cause) { // 1) New state - partition discovery loop executed as separate thread, with this // thread running the main fetcher loop // 2) Old state - partition discovery is disabled and only the main fetcher loop is executed + if (discoveryIntervalMillis == PARTITION_DISCOVERY_DISABLED) { + kafkaFetcher.runFetchLoop(); + } else { + runWithPartitionDiscovery(); + } + } - if (discoveryIntervalMillis != PARTITION_DISCOVERY_DISABLED) { - final AtomicReference discoveryLoopErrorRef = new AtomicReference<>(); - this.discoveryLoopThread = new Thread(new Runnable() { - @Override - public void run() { - try { - // --------------------- partition discovery loop --------------------- + private void runWithPartitionDiscovery() throws Exception { + final AtomicReference discoveryLoopErrorRef = new AtomicReference<>(); + createAndStartDiscoveryLoop(discoveryLoopErrorRef); - List discoveredPartitions; + kafkaFetcher.runFetchLoop(); - // throughout the loop, we always eagerly check if we are still running before - // performing the next operation, so that we can escape the loop as soon as possible + // make sure that the partition discoverer is waked up so that + // the discoveryLoopThread exits + partitionDiscoverer.wakeup(); + joinDiscoveryLoopThread(); - while (running) { - if (LOG.isDebugEnabled()) { - LOG.debug("Consumer subtask {} is trying to discover new partitions ...", getRuntimeContext().getIndexOfThisSubtask()); - } + // rethrow any fetcher errors + final Exception discoveryLoopError = discoveryLoopErrorRef.get(); + if (discoveryLoopError != null) { + throw new RuntimeException(discoveryLoopError); + } + } - try { - discoveredPartitions = partitionDiscoverer.discoverPartitions(); - } catch (AbstractPartitionDiscoverer.WakeupException | AbstractPartitionDiscoverer.ClosedException e) { - // the partition discoverer may have been closed or woken up before or during the discovery; - // this would only happen if the consumer was canceled; simply escape the loop - break; - } + @VisibleForTesting + void joinDiscoveryLoopThread() throws InterruptedException { + if (discoveryLoopThread != null) { + discoveryLoopThread.join(); + } + } - // no need to add the discovered partitions if we were closed during the meantime - if (running && !discoveredPartitions.isEmpty()) { - kafkaFetcher.addDiscoveredPartitions(discoveredPartitions); - } + private void createAndStartDiscoveryLoop(AtomicReference discoveryLoopErrorRef) { + discoveryLoopThread = new Thread(() -> { + try { + // --------------------- partition discovery loop --------------------- - // do not waste any time sleeping if we're not running anymore - if (running && discoveryIntervalMillis != 0) { - try { - Thread.sleep(discoveryIntervalMillis); - } catch (InterruptedException iex) { - // may be interrupted if the consumer was canceled midway; simply escape the loop - break; - } - } - } - } catch (Exception e) { - discoveryLoopErrorRef.set(e); - } finally { - // calling cancel will also let the fetcher loop escape - // (if not running, cancel() was already called) - if (running) { - cancel(); - } - } - } - }, "Kafka Partition Discovery for " + getRuntimeContext().getTaskNameWithSubtasks()); + // throughout the loop, we always eagerly check if we are still running before + // performing the next operation, so that we can escape the loop as soon as possible - discoveryLoopThread.start(); - kafkaFetcher.runFetchLoop(); + while (running) { + if (LOG.isDebugEnabled()) { + LOG.debug("Consumer subtask {} is trying to discover new partitions ...", getRuntimeContext().getIndexOfThisSubtask()); + } - // -------------------------------------------------------------------- + final List discoveredPartitions; + try { + discoveredPartitions = partitionDiscoverer.discoverPartitions(); + } catch (AbstractPartitionDiscoverer.WakeupException | AbstractPartitionDiscoverer.ClosedException e) { + // the partition discoverer may have been closed or woken up before or during the discovery; + // this would only happen if the consumer was canceled; simply escape the loop + break; + } - // make sure that the partition discoverer is properly closed - partitionDiscoverer.close(); - discoveryLoopThread.join(); + // no need to add the discovered partitions if we were closed during the meantime + if (running && !discoveredPartitions.isEmpty()) { + kafkaFetcher.addDiscoveredPartitions(discoveredPartitions); + } - // rethrow any fetcher errors - final Exception discoveryLoopError = discoveryLoopErrorRef.get(); - if (discoveryLoopError != null) { - throw new RuntimeException(discoveryLoopError); + // do not waste any time sleeping if we're not running anymore + if (running && discoveryIntervalMillis != 0) { + try { + Thread.sleep(discoveryIntervalMillis); + } catch (InterruptedException iex) { + // may be interrupted if the consumer was canceled midway; simply escape the loop + break; + } + } + } + } catch (Exception e) { + discoveryLoopErrorRef.set(e); + } finally { + // calling cancel will also let the fetcher loop escape + // (if not running, cancel() was already called) + if (running) { + cancel(); + } } - } else { - // won't be using the discoverer - partitionDiscoverer.close(); + }, "Kafka Partition Discovery for " + getRuntimeContext().getTaskNameWithSubtasks()); - kafkaFetcher.runFetchLoop(); - } + discoveryLoopThread.start(); } @Override @@ -766,11 +771,27 @@ public void cancel() { @Override public void close() throws Exception { - // pretty much the same logic as cancelling + cancel(); + + joinDiscoveryLoopThread(); + + Exception exception = null; + if (partitionDiscoverer != null) { + try { + partitionDiscoverer.close(); + } catch (Exception e) { + exception = e; + } + } + try { - cancel(); - } finally { super.close(); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } + + if (exception != null) { + throw exception; } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index dbe76302ab0a8..ff00b04a9f20f 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -47,12 +47,18 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.connectors.kafka.testutils.TestPartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.testutils.TestSourceContext; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.MockStreamingRuntimeContext; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.SupplierWithException; +import org.apache.flink.util.function.ThrowingRunnable; import org.junit.Assert; import org.junit.Test; @@ -70,10 +76,12 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.hamcrest.collection.IsIn.isIn; import static org.hamcrest.collection.IsMapContaining.hasKey; import static org.hamcrest.core.IsNot.not; @@ -83,12 +91,13 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; /** * Tests for the {@link FlinkKafkaConsumerBase}. */ -public class FlinkKafkaConsumerBaseTest { +public class FlinkKafkaConsumerBaseTest extends TestLogger { /** * Tests that not both types of timestamp extractors / watermark generators can be used. @@ -208,13 +217,7 @@ public void testConfigureAutoCommitMode() throws Exception { @SuppressWarnings("unchecked") final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); - setupConsumer( - consumer, - false, - null, - false, // disable checkpointing; auto commit should be respected - 0, - 1); + setupConsumer(consumer); assertEquals(OffsetCommitMode.KAFKA_PERIODIC, consumer.getOffsetCommitMode()); } @@ -242,13 +245,7 @@ public void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Except @SuppressWarnings("unchecked") final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(false); - setupConsumer( - consumer, - false, - null, - false, // disable checkpointing; auto commit should be respected - 0, - 1); + setupConsumer(consumer); assertEquals(OffsetCommitMode.DISABLED, consumer.getOffsetCommitMode()); } @@ -464,6 +461,98 @@ public void go() throws Exception { runThread.sync(); } + @Test + public void testClosePartitionDiscovererWhenOpenThrowException() throws Exception { + final RuntimeException failureCause = new RuntimeException(new FlinkException("Test partition discoverer exception")); + final FailingPartitionDiscoverer failingPartitionDiscoverer = new FailingPartitionDiscoverer(failureCause); + + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(failingPartitionDiscoverer); + + try { + setupConsumer(consumer); + fail("Exception should be thrown in open method"); + } catch (RuntimeException e) { + assertThat(ExceptionUtils.findThrowable(e, t -> t.equals(failureCause)).isPresent(), is(true)); + } + consumer.close(); + assertTrue("partitionDiscoverer should be closed when consumer is closed", failingPartitionDiscoverer.isClosed()); + } + + @Test + public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exception { + final FlinkException failureCause = new FlinkException("Create Kafka fetcher failure."); + + final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>( + () -> { + throw failureCause; + }, + testPartitionDiscoverer, + 100L); + + setupConsumer(consumer); + + try { + consumer.run(new TestSourceContext<>()); + fail("Exception should be thrown in run method"); + } catch (Exception e) { + assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(failureCause)).isPresent(), is(true)); + } + consumer.close(); + assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); + } + + @Test + public void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception { + final FlinkException failureCause = new FlinkException("Run Kafka fetcher failure."); + + final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); + final AbstractFetcher mock = (AbstractFetcher) mock(AbstractFetcher.class); + doThrow(failureCause).when(mock).runFetchLoop(); + + final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(() -> mock, testPartitionDiscoverer, 100L); + + setupConsumer(consumer); + + try { + consumer.run(new TestSourceContext<>()); + fail("Exception should be thrown in run method"); + } catch (Exception e) { + assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(failureCause)).isPresent(), is(true)); + } + consumer.close(); + consumer.joinDiscoveryLoopThread(); + assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); + } + + @Test + public void testClosePartitionDiscovererWithCancellation() throws Exception { + final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); + + final TestingFlinkKafkaConsumer consumer = new TestingFlinkKafkaConsumer<>(testPartitionDiscoverer, 100L); + + setupConsumer(consumer); + + CompletableFuture runFuture = CompletableFuture.runAsync(ThrowingRunnable.unchecked(() -> consumer.run(new TestSourceContext<>()))); + + consumer.close(); + + consumer.joinDiscoveryLoopThread(); + runFuture.get(); + + assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); + } + + protected void setupConsumer(FlinkKafkaConsumerBase consumer) throws Exception { + setupConsumer( + consumer, + false, + null, + false, + 0, + 1); + } + @Test public void testScaleUp() throws Exception { testRescaling(5, 2, 8, 30); @@ -607,6 +696,140 @@ private static AbstractStreamOperatorTestHarness createTestHarness( // ------------------------------------------------------------------------ + /** + * A dummy partition discoverer that always throws an exception from discoverPartitions() method. + */ + private static class FailingPartitionDiscoverer extends AbstractPartitionDiscoverer { + + private volatile boolean closed = false; + + private final RuntimeException failureCause; + + public FailingPartitionDiscoverer(RuntimeException failureCause) { + super( + new KafkaTopicsDescriptor(Arrays.asList("foo"), null), + 0, + 1); + this.failureCause = failureCause; + } + + @Override + protected void initializeConnections() throws Exception { + closed = false; + } + + @Override + protected void wakeupConnections() { + + } + + @Override + protected void closeConnections() throws Exception { + closed = true; + } + + @Override + protected List getAllTopics() throws WakeupException { + return null; + } + + @Override + protected List getAllPartitionsForTopics(List topics) throws WakeupException { + return null; + } + + @Override public List discoverPartitions() throws WakeupException, ClosedException { + throw failureCause; + } + + public boolean isClosed() { + return closed; + } + } + + private static class DummyPartitionDiscoverer extends AbstractPartitionDiscoverer { + + private final List allTopics; + private final List allPartitions; + private volatile boolean closed = false; + private volatile boolean wakedUp = false; + + private DummyPartitionDiscoverer() { + super(new KafkaTopicsDescriptor(Collections.singletonList("foo"), null), 0, 1); + this.allTopics = Collections.singletonList("foo"); + this.allPartitions = Collections.singletonList(new KafkaTopicPartition("foo", 0)); + } + + @Override + protected void initializeConnections() { + //noop + } + + @Override + protected void wakeupConnections() { + wakedUp = true; + } + + @Override + protected void closeConnections() { + closed = true; + } + + @Override + protected List getAllTopics() throws WakeupException { + checkState(); + + return allTopics; + } + + @Override + protected List getAllPartitionsForTopics(List topics) throws WakeupException { + checkState(); + return allPartitions; + } + + private void checkState() throws WakeupException { + if (wakedUp || closed) { + throw new WakeupException(); + } + } + + boolean isClosed() { + return closed; + } + } + + private static class TestingFetcher extends AbstractFetcher { + + private volatile boolean isRunning = true; + + protected TestingFetcher(SourceFunction.SourceContext sourceContext, Map seedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, ProcessingTimeService processingTimeProvider, long autoWatermarkInterval, ClassLoader userCodeClassLoader, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + super(sourceContext, seedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, processingTimeProvider, autoWatermarkInterval, userCodeClassLoader, consumerMetricGroup, useMetrics); + } + + @Override + public void runFetchLoop() throws Exception { + while (isRunning) { + Thread.sleep(10L); + } + } + + @Override + public void cancel() { + isRunning = false; + } + + @Override + protected void doCommitInternalOffsetsToKafka(Map offsets, @Nonnull KafkaCommitCallback commitCallback) throws Exception { + + } + + @Override + protected KPH createKafkaPartitionHandle(KafkaTopicPartition partition) { + return null; + } + } + /** * An instantiable dummy {@link FlinkKafkaConsumerBase} that supports injecting * mocks for {@link FlinkKafkaConsumerBase#kafkaFetcher}, {@link FlinkKafkaConsumerBase#partitionDiscoverer}, @@ -615,7 +838,7 @@ private static AbstractStreamOperatorTestHarness createTestHarness( private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase { private static final long serialVersionUID = 1L; - private AbstractFetcher testFetcher; + private SupplierWithException, Exception> testFetcherSupplier; private AbstractPartitionDiscoverer testPartitionDiscoverer; private boolean isAutoCommitEnabled; @@ -629,20 +852,56 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase, Exception> abstractFetcherSupplier, AbstractPartitionDiscoverer abstractPartitionDiscoverer, long discoveryIntervalMillis) { + this(abstractFetcherSupplier, abstractPartitionDiscoverer, false, discoveryIntervalMillis); + } + @SuppressWarnings("unchecked") DummyFlinkKafkaConsumer( AbstractFetcher testFetcher, AbstractPartitionDiscoverer testPartitionDiscoverer, boolean isAutoCommitEnabled) { + this( + testFetcher, + testPartitionDiscoverer, + isAutoCommitEnabled, + PARTITION_DISCOVERY_DISABLED); + } + + @SuppressWarnings("unchecked") + DummyFlinkKafkaConsumer( + AbstractFetcher testFetcher, + AbstractPartitionDiscoverer testPartitionDiscoverer, + boolean isAutoCommitEnabled, + long discoveryIntervalMillis) { + this( + () -> testFetcher, + testPartitionDiscoverer, + isAutoCommitEnabled, + discoveryIntervalMillis); + } + + @SuppressWarnings("unchecked") + DummyFlinkKafkaConsumer( + SupplierWithException, Exception> testFetcherSupplier, + AbstractPartitionDiscoverer testPartitionDiscoverer, + boolean isAutoCommitEnabled, + long discoveryIntervalMillis) { super( Collections.singletonList("dummy-topic"), null, (KeyedDeserializationSchema < T >) mock(KeyedDeserializationSchema.class), - PARTITION_DISCOVERY_DISABLED, + discoveryIntervalMillis, false); - this.testFetcher = testFetcher; + this.testFetcherSupplier = testFetcherSupplier; this.testPartitionDiscoverer = testPartitionDiscoverer; this.isAutoCommitEnabled = isAutoCommitEnabled; } @@ -658,7 +917,7 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase fetchOffsetsWithTimestamp( } } + private static class TestingFlinkKafkaConsumer extends FlinkKafkaConsumerBase { + + private static final long serialVersionUID = 935384661907656996L; + + private final AbstractPartitionDiscoverer partitionDiscoverer; + + TestingFlinkKafkaConsumer(final AbstractPartitionDiscoverer partitionDiscoverer, long discoveryIntervalMillis) { + super(Collections.singletonList("dummy-topic"), + null, + (KeyedDeserializationSchema < T >) mock(KeyedDeserializationSchema.class), + discoveryIntervalMillis, + false); + this.partitionDiscoverer = partitionDiscoverer; + } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map thisSubtaskPartitionsWithStartOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + return new TestingFetcher(sourceContext, thisSubtaskPartitionsWithStartOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext.getProcessingTimeService(), 0L, getClass().getClassLoader(), consumerMetricGroup, useMetrics); + + } + + @Override + protected AbstractPartitionDiscoverer createPartitionDiscoverer(KafkaTopicsDescriptor topicsDescriptor, int indexOfThisSubtask, int numParallelSubtasks) { + return partitionDiscoverer; + } + + @Override + protected boolean getIsAutoCommitEnabled() { + return false; + } + + @Override + protected Map fetchOffsetsWithTimestamp(Collection partitions, long timestamp) { + throw new UnsupportedOperationException("fetchOffsetsWithTimestamp is not supported"); + } + } + private static final class TestingListState implements ListState { private final List list = new ArrayList<>(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java index d024c04f5e018..655fe092823d8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java @@ -25,6 +25,8 @@ import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import java.util.Collections; @@ -79,6 +81,8 @@ public int getNumberOfParallelSubtasks() { private static class MockStreamOperator extends AbstractStreamOperator { private static final long serialVersionUID = -1153976702711944427L; + private transient TestProcessingTimeService testProcessingTimeService; + @Override public ExecutionConfig getExecutionConfig() { return new ExecutionConfig(); @@ -88,5 +92,13 @@ public ExecutionConfig getExecutionConfig() { public OperatorID getOperatorID() { return new OperatorID(); } + + @Override + protected ProcessingTimeService getProcessingTimeService() { + if (testProcessingTimeService == null) { + testProcessingTimeService = new TestProcessingTimeService(); + } + return testProcessingTimeService; + } } } From 4789b3f76857f31bf683589025500e43e1d99d69 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 29 Jan 2019 11:45:27 +0800 Subject: [PATCH 298/359] [FLINK-10774] [tests] Refactor Kafka tests to have consistent life cycle verifications --- .../kafka/FlinkKafkaConsumerBaseTest.java | 64 ++++++++----------- 1 file changed, 28 insertions(+), 36 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index ff00b04a9f20f..486a2860a0ca1 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -64,6 +64,7 @@ import org.junit.Test; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.Serializable; import java.util.ArrayDeque; @@ -468,13 +469,7 @@ public void testClosePartitionDiscovererWhenOpenThrowException() throws Exceptio final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(failingPartitionDiscoverer); - try { - setupConsumer(consumer); - fail("Exception should be thrown in open method"); - } catch (RuntimeException e) { - assertThat(ExceptionUtils.findThrowable(e, t -> t.equals(failureCause)).isPresent(), is(true)); - } - consumer.close(); + testConsumerLifeCycle(consumer, failureCause); assertTrue("partitionDiscoverer should be closed when consumer is closed", failingPartitionDiscoverer.isClosed()); } @@ -490,15 +485,7 @@ public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exc testPartitionDiscoverer, 100L); - setupConsumer(consumer); - - try { - consumer.run(new TestSourceContext<>()); - fail("Exception should be thrown in run method"); - } catch (Exception e) { - assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(failureCause)).isPresent(), is(true)); - } - consumer.close(); + testConsumerLifeCycle(consumer, failureCause); assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); } @@ -512,16 +499,7 @@ public void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(() -> mock, testPartitionDiscoverer, 100L); - setupConsumer(consumer); - - try { - consumer.run(new TestSourceContext<>()); - fail("Exception should be thrown in run method"); - } catch (Exception e) { - assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(failureCause)).isPresent(), is(true)); - } - consumer.close(); - consumer.joinDiscoveryLoopThread(); + testConsumerLifeCycle(consumer, failureCause); assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); } @@ -531,19 +509,33 @@ public void testClosePartitionDiscovererWithCancellation() throws Exception { final TestingFlinkKafkaConsumer consumer = new TestingFlinkKafkaConsumer<>(testPartitionDiscoverer, 100L); - setupConsumer(consumer); - - CompletableFuture runFuture = CompletableFuture.runAsync(ThrowingRunnable.unchecked(() -> consumer.run(new TestSourceContext<>()))); - - consumer.close(); - - consumer.joinDiscoveryLoopThread(); - runFuture.get(); - + testConsumerLifeCycle(consumer, null); assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); } - protected void setupConsumer(FlinkKafkaConsumerBase consumer) throws Exception { + private void testConsumerLifeCycle( + FlinkKafkaConsumerBase testKafkaConsumer, + @Nullable Exception expectedException) throws Exception { + + if (expectedException == null) { + setupConsumer(testKafkaConsumer); + final CompletableFuture runFuture = CompletableFuture.runAsync(ThrowingRunnable.unchecked(() -> testKafkaConsumer.run(new TestSourceContext<>()))); + testKafkaConsumer.close(); + runFuture.get(); + } else { + try { + setupConsumer(testKafkaConsumer); + testKafkaConsumer.run(new TestSourceContext<>()); + + fail("Exception should have been thrown from open / run method of FlinkKafkaConsumerBase."); + } catch (Exception e) { + assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(expectedException)).isPresent(), is(true)); + } + testKafkaConsumer.close(); + } + } + + private void setupConsumer(FlinkKafkaConsumerBase consumer) throws Exception { setupConsumer( consumer, false, From 198c4001fcce5fc4acc5fc85736b61f0ff4bd7e5 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 29 Jan 2019 12:10:55 +0800 Subject: [PATCH 299/359] [FLINK-10774] [tests] Test that Kafka partition discoverer is wokeup before closed when concurrently accessed --- .../kafka/FlinkKafkaConsumerBaseTest.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 486a2860a0ca1..b9924e403e2b6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -493,7 +493,11 @@ public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exc public void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception { final FlinkException failureCause = new FlinkException("Run Kafka fetcher failure."); - final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); + // in this scenario, the partition discoverer will be concurrently accessed; + // use the WakeupBeforeCloseTestingPartitionDiscoverer to verify that we always call + // wakeup() before closing the discoverer + final WakeupBeforeCloseTestingPartitionDiscoverer testPartitionDiscoverer = new WakeupBeforeCloseTestingPartitionDiscoverer(); + final AbstractFetcher mock = (AbstractFetcher) mock(AbstractFetcher.class); doThrow(failureCause).when(mock).runFetchLoop(); @@ -739,6 +743,17 @@ public boolean isClosed() { } } + private static class WakeupBeforeCloseTestingPartitionDiscoverer extends DummyPartitionDiscoverer { + @Override + protected void closeConnections() { + if (!isWakedUp()) { + fail("Partition discoverer should have been waked up first before closing."); + } + + super.closeConnections(); + } + } + private static class DummyPartitionDiscoverer extends AbstractPartitionDiscoverer { private final List allTopics; @@ -789,6 +804,10 @@ private void checkState() throws WakeupException { boolean isClosed() { return closed; } + + public boolean isWakedUp() { + return wakedUp; + } } private static class TestingFetcher extends AbstractFetcher { From 8012ab17bc86712f3bea3b641e4f39baf8c2485e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 30 Jan 2019 17:43:20 +0100 Subject: [PATCH 300/359] [FLINK-10774][tests] Refactor FlinkKafkaConsumerBaseTest#testConsumerLifeCycle Split #testConsumerLifeCycle into two methods which represent the two if-else branches. This closes #7606. --- .../kafka/FlinkKafkaConsumerBaseTest.java | 46 +++++++++---------- 1 file changed, 21 insertions(+), 25 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index b9924e403e2b6..1775c4c455384 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -64,7 +64,6 @@ import org.junit.Test; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.Serializable; import java.util.ArrayDeque; @@ -469,7 +468,7 @@ public void testClosePartitionDiscovererWhenOpenThrowException() throws Exceptio final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(failingPartitionDiscoverer); - testConsumerLifeCycle(consumer, failureCause); + testFailingConsumerLifecycle(consumer, failureCause); assertTrue("partitionDiscoverer should be closed when consumer is closed", failingPartitionDiscoverer.isClosed()); } @@ -485,7 +484,7 @@ public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exc testPartitionDiscoverer, 100L); - testConsumerLifeCycle(consumer, failureCause); + testFailingConsumerLifecycle(consumer, failureCause); assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); } @@ -503,40 +502,37 @@ public void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(() -> mock, testPartitionDiscoverer, 100L); - testConsumerLifeCycle(consumer, failureCause); + testFailingConsumerLifecycle(consumer, failureCause); assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); } + private void testFailingConsumerLifecycle(FlinkKafkaConsumerBase testKafkaConsumer, @Nonnull Exception expectedException) throws Exception { + try { + setupConsumer(testKafkaConsumer); + testKafkaConsumer.run(new TestSourceContext<>()); + + fail("Exception should have been thrown from open / run method of FlinkKafkaConsumerBase."); + } catch (Exception e) { + assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(expectedException)).isPresent(), is(true)); + } + testKafkaConsumer.close(); + } + @Test public void testClosePartitionDiscovererWithCancellation() throws Exception { final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); final TestingFlinkKafkaConsumer consumer = new TestingFlinkKafkaConsumer<>(testPartitionDiscoverer, 100L); - testConsumerLifeCycle(consumer, null); + testNormalConsumerLifecycle(consumer); assertTrue("partitionDiscoverer should be closed when consumer is closed", testPartitionDiscoverer.isClosed()); } - private void testConsumerLifeCycle( - FlinkKafkaConsumerBase testKafkaConsumer, - @Nullable Exception expectedException) throws Exception { - - if (expectedException == null) { - setupConsumer(testKafkaConsumer); - final CompletableFuture runFuture = CompletableFuture.runAsync(ThrowingRunnable.unchecked(() -> testKafkaConsumer.run(new TestSourceContext<>()))); - testKafkaConsumer.close(); - runFuture.get(); - } else { - try { - setupConsumer(testKafkaConsumer); - testKafkaConsumer.run(new TestSourceContext<>()); - - fail("Exception should have been thrown from open / run method of FlinkKafkaConsumerBase."); - } catch (Exception e) { - assertThat(ExceptionUtils.findThrowable(e, throwable -> throwable.equals(expectedException)).isPresent(), is(true)); - } - testKafkaConsumer.close(); - } + private void testNormalConsumerLifecycle(FlinkKafkaConsumerBase testKafkaConsumer) throws Exception { + setupConsumer(testKafkaConsumer); + final CompletableFuture runFuture = CompletableFuture.runAsync(ThrowingRunnable.unchecked(() -> testKafkaConsumer.run(new TestSourceContext<>()))); + testKafkaConsumer.close(); + runFuture.get(); } private void setupConsumer(FlinkKafkaConsumerBase consumer) throws Exception { From e1e10163da39030064e4c5ad06c6c645345b619c Mon Sep 17 00:00:00 2001 From: Konstantin Knauf Date: Thu, 31 Jan 2019 13:04:35 +0100 Subject: [PATCH 301/359] [FLINK-11473][metrics][docs] Clarify documentation on Latency Tracking --- docs/monitoring/metrics.md | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index bfa0576c98caf..d9c8bc208deb8 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1627,16 +1627,18 @@ bypassing them. In particular the markers are not accounting for the time record Only if operators are not able to accept new records, thus they are queuing up, the latency measured using the markers will reflect that. -All intermediate operators keep a list of the last `n` latencies from each source to compute -a latency distribution. -The sink operators keep a list from each source, and each parallel source instance to allow detecting -latency issues caused by individual machines. +The `LatencyMarker`s are used to derive a distribution of the latency between the sources of the topology and each +downstream operator. These distributions are reported as histogram metrics. The granularity of these distributions can +be controlled in the [Flink configuration]({{ site.baseurl }}/ops/config.html#metrics-latency-interval. For the highest +granularity `subtask` Flink will derive the latency distribution between every source subtask and every downstream +subtask, which results in quadratic (in the terms of the parallelism) number of histograms. Currently, Flink assumes that the clocks of all machines in the cluster are in sync. We recommend setting up an automated clock synchronisation service (like NTP) to avoid false latency results. Warning Enabling latency metrics can significantly impact the performance -of the cluster. It is highly recommended to only use them for debugging purposes. +of the cluster (in particular for `subtask` granularity). It is highly recommended to only use them for debugging +purposes. ## REST API integration From d44c578df0eccf064e30142e06acd7049d59538b Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 31 Jan 2019 14:53:58 +0100 Subject: [PATCH 302/359] [hotfix][docs] Bump latest stable version to 1.7.1 --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 93c67d0e88be3..bfa24aefa1683 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -27,7 +27,7 @@ # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) -version: "1.7.0" +version: "1.7.1" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version version_title: "1.7" From a5c6102bf642a0bfeff4438921aad3ab8d16cf1c Mon Sep 17 00:00:00 2001 From: "park.yq" Date: Fri, 18 Jan 2019 16:40:30 +0800 Subject: [PATCH 303/359] [FLINK-11389] Fix Incorrectly use job information when call getSerializedTaskInformation in class TaskDeploymentDescriptor --- .../deployment/TaskDeploymentDescriptor.java | 6 +++--- .../TaskDeploymentDescriptorTest.java | 20 +++++++++++++++++++ 2 files changed, 23 insertions(+), 3 deletions(-) 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 4f5b2311a24a2..bb038eb80a6ac 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 @@ -208,10 +208,10 @@ public SerializedValue getSerializedJobInformation() { */ @Nullable public SerializedValue getSerializedTaskInformation() { - if (serializedJobInformation instanceof NonOffloaded) { - NonOffloaded jobInformation = + if (serializedTaskInformation instanceof NonOffloaded) { + NonOffloaded taskInformation = (NonOffloaded) serializedTaskInformation; - return jobInformation.serializedValue; + return taskInformation.serializedValue; } else { throw new IllegalStateException( "Trying to work with offloaded serialized job information."); 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 e20d34b1d3e1f..22e943bbc1c2d 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 @@ -42,6 +42,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -110,6 +111,25 @@ public void testSerialization() { assertEquals(orig.getTaskRestore().getTaskStateSnapshot(), copy.getTaskRestore().getTaskStateSnapshot()); assertEquals(orig.getProducedPartitions(), copy.getProducedPartitions()); assertEquals(orig.getInputGates(), copy.getInputGates()); + + final TaskDeploymentDescriptor testOffLoadedTaskInformation = new TaskDeploymentDescriptor( + jobID, + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.Offloaded<>(new PermanentBlobKey()), + execId, + allocationId, + indexInSubtaskGroup, + attemptNumber, + targetSlotNumber, + taskRestore, + producedResults, + inputGates); + try { + testOffLoadedTaskInformation.getSerializedTaskInformation(); + } catch (Exception e) { + assertTrue(e instanceof IllegalStateException); + } + } catch (Exception e) { e.printStackTrace(); From c39192a21098882338f0dcd9636f9241814099ce Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 31 Jan 2019 11:39:55 +0100 Subject: [PATCH 304/359] [FLINK-11389][tests] Refactor TaskDeploymentDescriptorTest This closes #7532. --- .../TaskDeploymentDescriptorTest.java | 178 +++++++++--------- 1 file changed, 93 insertions(+), 85 deletions(-) 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 22e943bbc1c2d..a617ce1e5152d 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 @@ -33,107 +33,115 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.BatchTask; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; import org.junit.Test; +import javax.annotation.Nonnull; + +import java.io.IOException; import java.net.URL; import java.util.ArrayList; import java.util.List; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; /** * Tests for the {@link TaskDeploymentDescriptor}. */ -public class TaskDeploymentDescriptorTest { +public class TaskDeploymentDescriptorTest extends TestLogger { + + private static final JobID jobID = new JobID(); + private static final JobVertexID vertexID = new JobVertexID(); + private static final ExecutionAttemptID execId = new ExecutionAttemptID(); + private static final AllocationID allocationId = new AllocationID(); + private static final String jobName = "job name"; + private static final String taskName = "task name"; + private static final int numberOfKeyGroups = 1; + private static final int indexInSubtaskGroup = 0; + private static final int currentNumberOfSubtasks = 1; + private static final int attemptNumber = 0; + private static final Configuration jobConfiguration = new Configuration(); + private static final Configuration taskConfiguration = new Configuration(); + private static final Class invokableClass = BatchTask.class; + private static final List producedResults = new ArrayList(0); + private static final List inputGates = new ArrayList(0); + private static final List requiredJars = new ArrayList<>(0); + private static final List requiredClasspaths = new ArrayList<>(0); + private static final int targetSlotNumber = 47; + private static final TaskStateSnapshot taskStateHandles = new TaskStateSnapshot(); + private static final JobManagerTaskRestore taskRestore = new JobManagerTaskRestore(1L, taskStateHandles); + + private final SerializedValue executionConfig = new SerializedValue<>(new ExecutionConfig()); + private final SerializedValue serializedJobInformation = new SerializedValue<>(new JobInformation( + jobID, jobName, executionConfig, jobConfiguration, requiredJars, requiredClasspaths)); + private final SerializedValue serializedJobVertexInformation = new SerializedValue<>(new TaskInformation( + vertexID, taskName, currentNumberOfSubtasks, numberOfKeyGroups, invokableClass.getName(), taskConfiguration)); + + public TaskDeploymentDescriptorTest() throws IOException {} + @Test - public void testSerialization() { - try { - final JobID jobID = new JobID(); - final JobVertexID vertexID = new JobVertexID(); - final ExecutionAttemptID execId = new ExecutionAttemptID(); - final AllocationID allocationId = new AllocationID(); - final String jobName = "job name"; - final String taskName = "task name"; - final int numberOfKeyGroups = 1; - final int indexInSubtaskGroup = 0; - final int currentNumberOfSubtasks = 1; - final int attemptNumber = 0; - final Configuration jobConfiguration = new Configuration(); - final Configuration taskConfiguration = new Configuration(); - final Class invokableClass = BatchTask.class; - final List producedResults = new ArrayList(0); - final List inputGates = new ArrayList(0); - final List requiredJars = new ArrayList<>(0); - final List requiredClasspaths = new ArrayList<>(0); - final SerializedValue executionConfig = new SerializedValue<>(new ExecutionConfig()); - final SerializedValue serializedJobInformation = new SerializedValue<>(new JobInformation( - jobID, jobName, executionConfig, jobConfiguration, requiredJars, requiredClasspaths)); - final SerializedValue serializedJobVertexInformation = new SerializedValue<>(new TaskInformation( - vertexID, taskName, currentNumberOfSubtasks, numberOfKeyGroups, invokableClass.getName(), taskConfiguration)); - final int targetSlotNumber = 47; - final TaskStateSnapshot taskStateHandles = new TaskStateSnapshot(); - final JobManagerTaskRestore taskRestore = new JobManagerTaskRestore(1L, taskStateHandles); - - final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor( - jobID, - new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), - new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobVertexInformation), - execId, - allocationId, - indexInSubtaskGroup, - attemptNumber, - targetSlotNumber, - taskRestore, - producedResults, - inputGates); - - final TaskDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); - - assertFalse(orig.getSerializedJobInformation() == copy.getSerializedJobInformation()); - assertFalse(orig.getSerializedTaskInformation() == copy.getSerializedTaskInformation()); - assertFalse(orig.getExecutionAttemptId() == copy.getExecutionAttemptId()); - assertFalse(orig.getTaskRestore() == copy.getTaskRestore()); - assertFalse(orig.getProducedPartitions() == copy.getProducedPartitions()); - assertFalse(orig.getInputGates() == copy.getInputGates()); - - assertEquals(orig.getSerializedJobInformation(), copy.getSerializedJobInformation()); - assertEquals(orig.getSerializedTaskInformation(), copy.getSerializedTaskInformation()); - assertEquals(orig.getExecutionAttemptId(), copy.getExecutionAttemptId()); - assertEquals(orig.getAllocationId(), copy.getAllocationId()); - assertEquals(orig.getSubtaskIndex(), copy.getSubtaskIndex()); - assertEquals(orig.getAttemptNumber(), copy.getAttemptNumber()); - assertEquals(orig.getTargetSlotNumber(), copy.getTargetSlotNumber()); - assertEquals(orig.getTaskRestore().getRestoreCheckpointId(), copy.getTaskRestore().getRestoreCheckpointId()); - assertEquals(orig.getTaskRestore().getTaskStateSnapshot(), copy.getTaskRestore().getTaskStateSnapshot()); - assertEquals(orig.getProducedPartitions(), copy.getProducedPartitions()); - assertEquals(orig.getInputGates(), copy.getInputGates()); - - final TaskDeploymentDescriptor testOffLoadedTaskInformation = new TaskDeploymentDescriptor( - jobID, - new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), - new TaskDeploymentDescriptor.Offloaded<>(new PermanentBlobKey()), - execId, - allocationId, - indexInSubtaskGroup, - attemptNumber, - targetSlotNumber, - taskRestore, - producedResults, - inputGates); - try { - testOffLoadedTaskInformation.getSerializedTaskInformation(); - } catch (Exception e) { - assertTrue(e instanceof IllegalStateException); - } + public void testSerialization() throws Exception { + final TaskDeploymentDescriptor orig = createTaskDeploymentDescriptor( + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobVertexInformation)); + final TaskDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); + + assertFalse(orig.getSerializedJobInformation() == copy.getSerializedJobInformation()); + assertFalse(orig.getSerializedTaskInformation() == copy.getSerializedTaskInformation()); + assertFalse(orig.getExecutionAttemptId() == copy.getExecutionAttemptId()); + assertFalse(orig.getTaskRestore() == copy.getTaskRestore()); + assertFalse(orig.getProducedPartitions() == copy.getProducedPartitions()); + assertFalse(orig.getInputGates() == copy.getInputGates()); + + assertEquals(orig.getSerializedJobInformation(), copy.getSerializedJobInformation()); + assertEquals(orig.getSerializedTaskInformation(), copy.getSerializedTaskInformation()); + assertEquals(orig.getExecutionAttemptId(), copy.getExecutionAttemptId()); + assertEquals(orig.getAllocationId(), copy.getAllocationId()); + assertEquals(orig.getSubtaskIndex(), copy.getSubtaskIndex()); + assertEquals(orig.getAttemptNumber(), copy.getAttemptNumber()); + assertEquals(orig.getTargetSlotNumber(), copy.getTargetSlotNumber()); + assertEquals(orig.getTaskRestore().getRestoreCheckpointId(), copy.getTaskRestore().getRestoreCheckpointId()); + assertEquals(orig.getTaskRestore().getTaskStateSnapshot(), copy.getTaskRestore().getTaskStateSnapshot()); + assertEquals(orig.getProducedPartitions(), copy.getProducedPartitions()); + assertEquals(orig.getInputGates(), copy.getInputGates()); + } + + @Test + public void testOffLoadedAndNonOffLoadedPayload() { + final TaskDeploymentDescriptor taskDeploymentDescriptor = createTaskDeploymentDescriptor( + new TaskDeploymentDescriptor.NonOffloaded<>(serializedJobInformation), + new TaskDeploymentDescriptor.Offloaded<>(new PermanentBlobKey())); + + SerializedValue actualSerializedJobInformation = taskDeploymentDescriptor.getSerializedJobInformation(); + assertThat(actualSerializedJobInformation, is(serializedJobInformation)); + + try { + taskDeploymentDescriptor.getSerializedTaskInformation(); + fail("Expected to fail since the task information should be offloaded."); + } catch (IllegalStateException expected) { + // expected } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + } + + @Nonnull + private TaskDeploymentDescriptor createTaskDeploymentDescriptor(TaskDeploymentDescriptor.MaybeOffloaded jobInformation, TaskDeploymentDescriptor.MaybeOffloaded taskInformation) { + return new TaskDeploymentDescriptor( + jobID, + jobInformation, + taskInformation, + execId, + allocationId, + indexInSubtaskGroup, + attemptNumber, + targetSlotNumber, + taskRestore, + producedResults, + inputGates); } } From e061cfef22afc3f41437a5ebb9cc726772c287d8 Mon Sep 17 00:00:00 2001 From: EAlexRojas Date: Mon, 28 Jan 2019 19:01:48 +0100 Subject: [PATCH 305/359] [FLINK-11419][filesystem] Wait until lease is revoked before truncating file in Hadoop. --- .../HadoopRecoverableFsDataOutputStream.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java index c688b3217c4ec..a716029c99e6f 100644 --- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java +++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java @@ -86,14 +86,21 @@ class HadoopRecoverableFsDataOutputStream extends RecoverableFsDataOutputStream this.targetFile = checkNotNull(recoverable.targetFile()); this.tempFile = checkNotNull(recoverable.tempFile()); + waitUntilLeaseIsRevoked(tempFile); + // truncate back and append + boolean truncated; try { - truncate(fs, tempFile, recoverable.offset()); + truncated = truncate(fs, tempFile, recoverable.offset()); } catch (Exception e) { throw new IOException("Missing data in tmp file: " + tempFile, e); } - waitUntilLeaseIsRevoked(tempFile); + if (!truncated) { + // Truncate did not complete immediately, we must wait for the operation to complete and release the lease + waitUntilLeaseIsRevoked(tempFile); + } + out = fs.append(tempFile); // sanity check @@ -173,10 +180,10 @@ private static void ensureTruncateInitialized() throws FlinkRuntimeException { } } - static void truncate(FileSystem hadoopFs, Path file, long length) throws IOException { + static boolean truncate(FileSystem hadoopFs, Path file, long length) throws IOException { if (truncateHandle != null) { try { - truncateHandle.invoke(hadoopFs, file, length); + return (Boolean) truncateHandle.invoke(hadoopFs, file, length); } catch (InvocationTargetException e) { ExceptionUtils.rethrowIOException(e.getTargetException()); @@ -190,6 +197,7 @@ static void truncate(FileSystem hadoopFs, Path file, long length) throws IOExcep else { throw new IllegalStateException("Truncation handle has not been initialized"); } + return true; } // ------------------------------------------------------------------------ From 11bbf952f0fa0a4b897c37027031dccebf11485e Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 31 Jan 2019 13:36:09 +0100 Subject: [PATCH 306/359] [FLINK-11419][filesystem] Wait for lease to be revoked when truncating file in Hadoop. --- .../HadoopRecoverableFsDataOutputStream.java | 52 +++++++++++-------- 1 file changed, 29 insertions(+), 23 deletions(-) diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java index a716029c99e6f..be0d134b54e1c 100644 --- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java +++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableFsDataOutputStream.java @@ -86,20 +86,7 @@ class HadoopRecoverableFsDataOutputStream extends RecoverableFsDataOutputStream this.targetFile = checkNotNull(recoverable.targetFile()); this.tempFile = checkNotNull(recoverable.tempFile()); - waitUntilLeaseIsRevoked(tempFile); - - // truncate back and append - boolean truncated; - try { - truncated = truncate(fs, tempFile, recoverable.offset()); - } catch (Exception e) { - throw new IOException("Missing data in tmp file: " + tempFile, e); - } - - if (!truncated) { - // Truncate did not complete immediately, we must wait for the operation to complete and release the lease - waitUntilLeaseIsRevoked(tempFile); - } + safelyTruncateFile(fs, tempFile, recoverable); out = fs.append(tempFile); @@ -162,6 +149,30 @@ public void close() throws IOException { // Hadoop 2.7, which have no truncation calls for HDFS. // ------------------------------------------------------------------------ + private static void safelyTruncateFile( + final FileSystem fileSystem, + final Path path, + final HadoopFsRecoverable recoverable) throws IOException { + + ensureTruncateInitialized(); + + waitUntilLeaseIsRevoked(fileSystem, path); + + // truncate back and append + boolean truncated; + try { + truncated = truncate(fileSystem, path, recoverable.offset()); + } catch (Exception e) { + throw new IOException("Problem while truncating file: " + path, e); + } + + if (!truncated) { + // Truncate did not complete immediately, we must wait for + // the operation to complete and release the lease. + waitUntilLeaseIsRevoked(fileSystem, path); + } + } + private static void ensureTruncateInitialized() throws FlinkRuntimeException { if (truncateHandle == null) { Method truncateMethod; @@ -180,7 +191,7 @@ private static void ensureTruncateInitialized() throws FlinkRuntimeException { } } - static boolean truncate(FileSystem hadoopFs, Path file, long length) throws IOException { + private static boolean truncate(final FileSystem hadoopFs, final Path file, final long length) throws IOException { if (truncateHandle != null) { try { return (Boolean) truncateHandle.invoke(hadoopFs, file, length); @@ -197,7 +208,7 @@ static boolean truncate(FileSystem hadoopFs, Path file, long length) throws IOEx else { throw new IllegalStateException("Truncation handle has not been initialized"); } - return true; + return false; } // ------------------------------------------------------------------------ @@ -268,12 +279,7 @@ public void commitAfterRecovery() throws IOException { if (srcStatus.getLen() > expectedLength) { // can happen if we go from persist to recovering for commit directly // truncate the trailing junk away - try { - truncate(fs, src, expectedLength); - } catch (Exception e) { - // this can happen if the file is smaller than expected - throw new IOException("Problem while truncating file: " + src, e); - } + safelyTruncateFile(fs, src, recoverable); } // rename to final location (if it exists, overwrite it) @@ -312,7 +318,7 @@ public CommitRecoverable getRecoverable() { * * @param path The path to the file we want to resume writing to. */ - private boolean waitUntilLeaseIsRevoked(final Path path) throws IOException { + private static boolean waitUntilLeaseIsRevoked(final FileSystem fs, final Path path) throws IOException { Preconditions.checkState(fs instanceof DistributedFileSystem); final DistributedFileSystem dfs = (DistributedFileSystem) fs; From d9c45af35eaf8827a6dd4d57d2f6522b96c542b0 Mon Sep 17 00:00:00 2001 From: xueyu <278006819@qq.com> Date: Mon, 21 Jan 2019 18:20:19 +0800 Subject: [PATCH 307/359] [FLINK-11046] [elasticsearch] Fix ElasticsearchSink deadlock when index failed with retry --- .../ElasticsearchApiCallBridge.java | 12 +++ .../ElasticsearchFailureHandlerIndexer.java | 88 +++++++++++++++++++ .../elasticsearch/ElasticsearchSinkBase.java | 19 +++- .../ElasticsearchSinkBaseTest.java | 6 +- .../tests/Elasticsearch6SinkExample.java | 50 ++++++++++- flink-end-to-end-tests/test-scripts/common.sh | 5 +- 6 files changed, 171 insertions(+), 9 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index d3b774c8428d4..e450485d6f57b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -93,6 +93,18 @@ default RequestIndexer createBulkProcessorIndexer( numPendingRequestsRef); } + /** + * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. + */ + default ElasticsearchFailureHandlerIndexer createFailureHandlerIndexer( + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new ElasticsearchFailureHandlerIndexer( + flushOnCheckpoint, + numPendingRequestsRef); + } + + /** * Perform any necessary state cleanup. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java new file mode 100644 index 0000000000000..685b5d9c602b9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java @@ -0,0 +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.streaming.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkRequest}. + * {@link ActionRequest ActionRequests} will be buffered before re-sending a bulk request to the Elasticsearch cluster. + */ + +@Internal +class ElasticsearchFailureHandlerIndexer implements RequestIndexer { + + private BulkRequest bulkRequest; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + ElasticsearchFailureHandlerIndexer(boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { + this.bulkRequest = new BulkRequest(); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkRequest.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkRequest.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkRequest.add(updateRequest); + } + } + + public BulkRequest getBulkRequest() { + return bulkRequest; + } + + public int numberOfActions() { + return bulkRequest.numberOfActions(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 4d0c00252d279..55d8854a3a929 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -166,6 +166,8 @@ public void setDelayMillis(long delayMillis) { /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient RequestIndexer requestIndexer; + private transient ElasticsearchFailureHandlerIndexer failureRequestIndexer; + // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink // ------------------------------------------------------------------------ @@ -296,12 +298,14 @@ public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = callBridge.createFailureHandlerIndexer(flushOnCheckpoint, numPendingRequests); } @Override public void invoke(T value) throws Exception { // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink checkErrorAndRethrow(); + reindexFailedRequest(); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } @@ -380,6 +384,15 @@ private void checkErrorAndRethrow() { } } + private void reindexFailedRequest() { + if (failureRequestIndexer.numberOfActions() > 0) { + BulkRequest failedRequest = failureRequestIndexer.getBulkRequest(); + for (ActionRequest request: failedRequest.requests()) { + requestIndexer.add(request); + } + } + } + private class BulkProcessorListener implements BulkProcessor.Listener { @Override public void beforeBulk(long executionId, BulkRequest request) { } @@ -400,9 +413,9 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon restStatus = itemResponse.getFailure().getStatus(); if (restStatus == null) { - failureHandler.onFailure(request.requests().get(i), failure, -1, requestIndexer); + failureHandler.onFailure(request.requests().get(i), failure, -1, failureRequestIndexer); } else { - failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), requestIndexer); + failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), failureRequestIndexer); } } } @@ -424,7 +437,7 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) try { for (ActionRequest action : request.requests()) { - failureHandler.onFailure(action, failure, -1, requestIndexer); + failureHandler.onFailure(action, failure, -1, failureRequestIndexer); } } catch (Throwable t) { // fail the sink and skip the rest of the items diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 322d64cbe0ebd..fdfdece9e5352 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -366,9 +366,9 @@ public void go() throws Exception { // since the previous flush should have resulted in a request re-add from the failure handler, // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd flush - while (snapshotThread.getState() != Thread.State.WAITING) { - Thread.sleep(10); - } +// while (snapshotThread.getState() != Thread.State.WAITING) { +// Thread.sleep(10); +// } // current number of pending request should be 1 due to the re-add Assert.assertEquals(1, sink.getNumPendingRequests()); diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java index e813c2995f5f7..6533ea220b482 100644 --- a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java @@ -23,11 +23,13 @@ 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.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; import org.apache.flink.util.Collector; import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.Requests; @@ -77,6 +79,9 @@ public void flatMap(Long value, Collector> out) { indexer.add(createUpdateRequest(element, parameterTool)); }); + esSinkBuilder.setFailureHandler( + new CustomFailureHandler(parameterTool.getRequired("index"), parameterTool.getRequired("type"))); + // this instructs the sink to emit after every element, otherwise they would be buffered esSinkBuilder.setBulkFlushMaxActions(1); @@ -85,13 +90,54 @@ public void flatMap(Long value, Collector> out) { env.execute("Elasticsearch 6.x end to end sink test example"); } + private static class CustomFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 942269087742453482L; + + private final String index; + private final String type; + + CustomFailureHandler(String index, String type) { + this.index = index; + this.type = type; + } + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { + if (action instanceof IndexRequest) { + Map json = new HashMap<>(); + json.put("data", ((IndexRequest) action).source()); + + indexer.add( + Requests.indexRequest() + .index(index) + .type(type) + .id(((IndexRequest) action).id()) + .source(json)); + } else { + throw new IllegalStateException("unexpected"); + } + } + } + private static IndexRequest createIndexRequest(String element, ParameterTool parameterTool) { Map json = new HashMap<>(); json.put("data", element); + String index; + String type; + + if (element.startsWith("message #15")) { + index = ":intentional invalid index:"; + type = ":intentional invalid type:"; + } else { + index = parameterTool.getRequired("index"); + type = parameterTool.getRequired("type"); + } + return Requests.indexRequest() - .index(parameterTool.getRequired("index")) - .type(parameterTool.getRequired("type")) + .index(index) + .type(type) .id(element) .source(json); } diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 645cf2a2f1e74..183735b0a1071 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -296,7 +296,7 @@ function check_logs_for_errors { | grep -v "AskTimeoutException" \ | grep -v "Error while loading kafka-version.properties" \ | grep -v "WARN akka.remote.transport.netty.NettyTransport" \ - | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ + | grep -v "WARN org.apache.flink.shaded.akka.org.jboss.netty.channel.DefaultChannelPipeline" \ | grep -v "jvm-exit-on-fatal-error" \ | grep -v '^INFO:.*AWSErrorCode=\[400 Bad Request\].*ServiceEndpoint=\[https://.*\.s3\.amazonaws\.com\].*RequestType=\[HeadBucketRequest\]' \ | grep -v "RejectedExecutionException" \ @@ -305,6 +305,7 @@ function check_logs_for_errors { | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \ | grep -v "org.apache.flink.fs.shaded.hadoop3.org.apache.commons.beanutils.FluentPropertyBeanIntrospector - Error when creating PropertyDescriptor for public final void org.apache.flink.fs.shaded.hadoop3.org.apache.commons.configuration2.AbstractConfiguration.setProperty(java.lang.String,java.lang.Object)! Ignoring this property." \ | grep -v "Error while loading kafka-version.properties :null" \ + | grep -v "Failed Elasticsearch item request" \ | grep -ic "error" || true) if [[ ${error_count} -gt 0 ]]; then echo "Found error in log files:" @@ -337,6 +338,8 @@ function check_logs_for_exceptions { | grep -v "Caused by: java.lang.Exception: JobManager is shutting down" \ | grep -v "java.lang.Exception: Artificial failure" \ | grep -v "org.apache.flink.runtime.checkpoint.decline" \ + | grep -v "org.elasticsearch.ElasticsearchException" \ + | grep -v "Elasticsearch exception" \ | grep -ic "exception" || true) if [[ ${exception_count} -gt 0 ]]; then echo "Found exception in log files:" From 35af99391dac431c85e30bcc98b89cba79bccfea Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 9 Feb 2019 18:11:25 +0800 Subject: [PATCH 308/359] [FLINK-11046] [elasticsearch] Simplify failure handler indexer implementation Since we always use the original request indexer to re-index failed requests, and that indexer already keeps track of the number of pending actions, the ElasticsearchFailureHandlerIndexer does not need to keep track of the number of pending records by itself. --- .../BufferingNoOpRequestIndexer.java | 76 ++++++++++++++++ .../ElasticsearchApiCallBridge.java | 12 --- .../ElasticsearchFailureHandlerIndexer.java | 88 ------------------- .../elasticsearch/ElasticsearchSinkBase.java | 20 ++--- 4 files changed, 82 insertions(+), 114 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java new file mode 100644 index 0000000000000..e639b820ae317 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java @@ -0,0 +1,76 @@ +/* + * 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.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Implementation of a {@link RequestIndexer} that buffers {@link ActionRequest ActionRequests} + * before re-sending them to the Elasticsearch cluster upon request. + */ +@Internal +@NotThreadSafe +class BufferingNoOpRequestIndexer implements RequestIndexer { + + private List bufferedRequests; + + BufferingNoOpRequestIndexer() { + this.bufferedRequests = new ArrayList<>(10); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + Collections.addAll(bufferedRequests, deleteRequests); + } + + @Override + public void add(IndexRequest... indexRequests) { + Collections.addAll(bufferedRequests, indexRequests); + } + + @Override + public void add(UpdateRequest... updateRequests) { + Collections.addAll(bufferedRequests, updateRequests); + } + + void processBufferedRequests(RequestIndexer actualIndexer) { + for (ActionRequest request : bufferedRequests) { + if (request instanceof IndexRequest) { + actualIndexer.add((IndexRequest) request); + } else if (request instanceof DeleteRequest) { + actualIndexer.add((DeleteRequest) request); + } else if (request instanceof UpdateRequest) { + actualIndexer.add((UpdateRequest) request); + } + } + + bufferedRequests.clear(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index e450485d6f57b..d3b774c8428d4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -93,18 +93,6 @@ default RequestIndexer createBulkProcessorIndexer( numPendingRequestsRef); } - /** - * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. - */ - default ElasticsearchFailureHandlerIndexer createFailureHandlerIndexer( - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - return new ElasticsearchFailureHandlerIndexer( - flushOnCheckpoint, - numPendingRequestsRef); - } - - /** * Perform any necessary state cleanup. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java deleted file mode 100644 index 685b5d9c602b9..0000000000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java +++ /dev/null @@ -1,88 +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.streaming.connectors.elasticsearch; - -import org.apache.flink.annotation.Internal; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; - -import java.util.concurrent.atomic.AtomicLong; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Implementation of a {@link RequestIndexer}, using a {@link BulkRequest}. - * {@link ActionRequest ActionRequests} will be buffered before re-sending a bulk request to the Elasticsearch cluster. - */ - -@Internal -class ElasticsearchFailureHandlerIndexer implements RequestIndexer { - - private BulkRequest bulkRequest; - private final boolean flushOnCheckpoint; - private final AtomicLong numPendingRequestsRef; - - ElasticsearchFailureHandlerIndexer(boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { - this.bulkRequest = new BulkRequest(); - this.flushOnCheckpoint = flushOnCheckpoint; - this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); - } - - @Override - public void add(DeleteRequest... deleteRequests) { - for (DeleteRequest deleteRequest : deleteRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkRequest.add(deleteRequest); - } - } - - @Override - public void add(IndexRequest... indexRequests) { - for (IndexRequest indexRequest : indexRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkRequest.add(indexRequest); - } - } - - @Override - public void add(UpdateRequest... updateRequests) { - for (UpdateRequest updateRequest : updateRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkRequest.add(updateRequest); - } - } - - public BulkRequest getBulkRequest() { - return bulkRequest; - } - - public int numberOfActions() { - return bulkRequest.numberOfActions(); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 55d8854a3a929..e50140ddfc9ef 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -166,7 +166,8 @@ public void setDelayMillis(long delayMillis) { /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient RequestIndexer requestIndexer; - private transient ElasticsearchFailureHandlerIndexer failureRequestIndexer; + /** Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed requests. */ + private transient BufferingNoOpRequestIndexer failureRequestIndexer; // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink @@ -298,15 +299,15 @@ public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); - failureRequestIndexer = callBridge.createFailureHandlerIndexer(flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = new BufferingNoOpRequestIndexer(); } @Override public void invoke(T value) throws Exception { // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink checkErrorAndRethrow(); - reindexFailedRequest(); + failureRequestIndexer.processBufferedRequests(requestIndexer); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } @@ -320,10 +321,10 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { checkErrorAndRethrow(); if (flushOnCheckpoint) { - do { + while (numPendingRequests.get() != 0) { bulkProcessor.flush(); checkErrorAndRethrow(); - } while (numPendingRequests.get() != 0); + } } } @@ -384,15 +385,6 @@ private void checkErrorAndRethrow() { } } - private void reindexFailedRequest() { - if (failureRequestIndexer.numberOfActions() > 0) { - BulkRequest failedRequest = failureRequestIndexer.getBulkRequest(); - for (ActionRequest request: failedRequest.requests()) { - requestIndexer.add(request); - } - } - } - private class BulkProcessorListener implements BulkProcessor.Listener { @Override public void beforeBulk(long executionId, BulkRequest request) { } From 2f522271abf03c5584612076b549c98d76a07f0f Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 9 Feb 2019 19:12:46 +0800 Subject: [PATCH 309/359] [FLINK-11046] [elasticsearch] Always re-index buffered failed requests on snapshot This ensures that at-least-once semantics is not broken due to the changes to the failure handle request indexer. This closes #7576. --- .../elasticsearch/ElasticsearchSinkBase.java | 14 ++++++++------ .../elasticsearch/ElasticsearchSinkBaseTest.java | 6 +++--- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index e50140ddfc9ef..5399c98e04003 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -304,10 +304,7 @@ public void open(Configuration parameters) throws Exception { @Override public void invoke(T value) throws Exception { - // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink - checkErrorAndRethrow(); - - failureRequestIndexer.processBufferedRequests(requestIndexer); + checkAsyncErrorsAndRequests(); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } @@ -318,12 +315,12 @@ public void initializeState(FunctionInitializationContext context) throws Except @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { - checkErrorAndRethrow(); + checkAsyncErrorsAndRequests(); if (flushOnCheckpoint) { while (numPendingRequests.get() != 0) { bulkProcessor.flush(); - checkErrorAndRethrow(); + checkAsyncErrorsAndRequests(); } } } @@ -385,6 +382,11 @@ private void checkErrorAndRethrow() { } } + private void checkAsyncErrorsAndRequests() { + checkErrorAndRethrow(); + failureRequestIndexer.processBufferedRequests(requestIndexer); + } + private class BulkProcessorListener implements BulkProcessor.Listener { @Override public void beforeBulk(long executionId, BulkRequest request) { } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index fdfdece9e5352..322d64cbe0ebd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -366,9 +366,9 @@ public void go() throws Exception { // since the previous flush should have resulted in a request re-add from the failure handler, // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd flush -// while (snapshotThread.getState() != Thread.State.WAITING) { -// Thread.sleep(10); -// } + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } // current number of pending request should be 1 due to the re-add Assert.assertEquals(1, sink.getNumPendingRequests()); From 6a02f74a54b3b9b977cae3e1f5975a17c6ace805 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 9 Feb 2019 19:15:32 +0800 Subject: [PATCH 310/359] [hotfix] [elasticsearch] Replace implementation of deprecated invoke method in ElasticsearchSinkBase --- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 5399c98e04003..da06d1c3a0878 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -203,7 +203,7 @@ public void setDelayMillis(long delayMillis) { private final AtomicReference failureThrowable = new AtomicReference<>(); public ElasticsearchSinkBase( - ElasticsearchApiCallBridge callBridge, + ElasticsearchApiCallBridge callBridge, Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction, ActionRequestFailureHandler failureHandler) { @@ -303,7 +303,7 @@ public void open(Configuration parameters) throws Exception { } @Override - public void invoke(T value) throws Exception { + public void invoke(T value, Context context) throws Exception { checkAsyncErrorsAndRequests(); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } From 22b60aa1d0ee3f06f5858c74f29c265cc3a687ab Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 12 Feb 2019 16:03:32 +0100 Subject: [PATCH 311/359] [hotfix][docs] Add space in self-closing linebreak tag --- .../apache/flink/configuration/description/HtmlFormatter.java | 2 +- .../flink/configuration/description/DescriptionHtmlTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java b/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java index a47530315c5c2..53f540ed63f1f 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/description/HtmlFormatter.java @@ -32,7 +32,7 @@ protected void formatLink(StringBuilder state, String link, String description) @Override protected void formatLineBreak(StringBuilder state) { - state.append("
    "); + state.append("
    "); } @Override diff --git a/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java b/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java index fcdb15583d647..bca827257efd1 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/description/DescriptionHtmlTest.java @@ -93,7 +93,7 @@ public void testDescriptionWithLineBreak() { String formattedDescription = new HtmlFormatter().format(description); assertEquals( - "This is first line.
    This is second line.", + "This is first line.
    This is second line.", formattedDescription); } From e64d3a8111b1faf2c06863fe144155e69ab27471 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 12 Feb 2019 16:03:54 +0100 Subject: [PATCH 312/359] [FLINK-11584][docs][tests] Fix linebreak parsing --- .../docs/configuration/ConfigOptionsDocsCompletenessITCase.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java index 25ce3dca4d1b3..bfc55c1ebbdd1 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocsCompletenessITCase.java @@ -171,6 +171,7 @@ private static Map parseDocumentedOptions() throws IOE private static Collection parseDocumentedOptionsFromFile(Path file) throws IOException { Document document = Jsoup.parse(file.toFile(), StandardCharsets.UTF_8.name()); + document.outputSettings().syntax(Document.OutputSettings.Syntax.xml); document.outputSettings().prettyPrint(false); return document.getElementsByTag("table").stream() .map(element -> element.getElementsByTag("tbody").get(0)) From 26abcfd4934d5ddae517f66f4485ad5d0c2f3593 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 12 Feb 2019 15:16:47 +0100 Subject: [PATCH 313/359] [FLINK-11585][docs] Fix prefix matching --- .../ConfigOptionsDocGenerator.java | 21 +++++--- .../ConfigOptionsDocGeneratorTest.java | 53 +++++++++++++++++++ 2 files changed, 66 insertions(+), 8 deletions(-) diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index 36a2203b7e4f7..ed8c2378ac0b3 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -332,11 +332,20 @@ List getDefaultOptions() { private Node findGroupRoot(String key) { String[] keyComponents = key.split("\\."); + Node lastRootNode = root; Node currentNode = root; for (String keyComponent : keyComponents) { - currentNode = currentNode.findChild(keyComponent); + final Node childNode = currentNode.getChild(keyComponent); + if (childNode == null) { + break; + } else { + currentNode = childNode; + if (currentNode.isGroupRoot()) { + lastRootNode = currentNode; + } + } } - return currentNode.isGroupRoot() ? currentNode : root; + return lastRootNode; } private static class Node { @@ -353,12 +362,8 @@ private Node addChild(String keyComponent) { return child; } - private Node findChild(String keyComponent) { - Node child = children.get(keyComponent); - if (child == null) { - return this; - } - return child; + private Node getChild(String keyComponent) { + return children.get(keyComponent); } private void assignOption(OptionWithMetaInfo option) { diff --git a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java index c0fb7c99eda73..438c10848b2b5 100644 --- a/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java +++ b/flink-docs/src/test/java/org/apache/flink/docs/configuration/ConfigOptionsDocGeneratorTest.java @@ -38,7 +38,10 @@ import java.nio.file.Paths; import java.util.HashMap; import java.util.List; +import java.util.Map; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; /** @@ -90,6 +93,56 @@ public void testCreatingDescription() { assertEquals(expectedTable, htmlTable); } + @ConfigGroups(groups = { + @ConfigGroup(name = "group1", keyPrefix = "a.b"), + @ConfigGroup(name = "group2", keyPrefix = "a.b.c.d")}) + static class TestConfigPrefix { + // should end up in the default group + public static ConfigOption option1 = ConfigOptions + .key("a.option") + .defaultValue(2); + + // should end up in group1, perfect key-prefix match + public static ConfigOption option2 = ConfigOptions + .key("a.b.option") + .noDefaultValue(); + + // should end up in group1, full key-prefix match + public static ConfigOption option3 = ConfigOptions + .key("a.b.c.option") + .defaultValue(2); + + // should end up in group1, full key-prefix match for group 1, partial match for group 2 + // checks that the generator remembers the last encountered root node + public static ConfigOption option4 = ConfigOptions + .key("a.b.c.e.option") + .defaultValue(2); + + // should end up in the default group, since no group exists with prefix "a.c" + // checks that the generator does not ignore components (like ignoring "c" to find a match "a.b") + public static ConfigOption option5 = ConfigOptions + .key("a.c.b.option") + .noDefaultValue(); + } + + @Test + public void testLongestPrefixMatching() { + final List> tables = ConfigOptionsDocGenerator.generateTablesForClass( + TestConfigPrefix.class); + + assertEquals(3, tables.size()); + final Map tablesConverted = new HashMap<>(tables.size()); + for (final Tuple2 table : tables) { + tablesConverted.put(table.f0 != null ? table.f0.name() : "default", table.f1); + } + + assertThat(tablesConverted.get("group1"), containsString("a.b.option")); + assertThat(tablesConverted.get("group1"), containsString("a.b.c.option")); + assertThat(tablesConverted.get("group1"), containsString("a.b.c.e.option")); + assertThat(tablesConverted.get("default"), containsString("a.option")); + assertThat(tablesConverted.get("default"), containsString("a.c.b.option")); + } + @ConfigGroups(groups = { @ConfigGroup(name = "firstGroup", keyPrefix = "first"), @ConfigGroup(name = "secondGroup", keyPrefix = "second")}) From 4c1907edb57f9c7c263a857d2aed7691f044f865 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 15 Feb 2019 11:33:36 +0100 Subject: [PATCH 314/359] [FLINK-11628][travis] Cache maven --- .travis.yml | 11 +++-------- tools/travis/setup_maven.sh | 38 +++++++++++++++++++++++++++++++++++++ tools/travis_controller.sh | 6 ++++++ 3 files changed, 47 insertions(+), 8 deletions(-) create mode 100755 tools/travis/setup_maven.sh diff --git a/.travis.yml b/.travis.yml index a7b1eebb4929d..49b4527427e76 100644 --- a/.travis.yml +++ b/.travis.yml @@ -25,6 +25,8 @@ cache: directories: - $HOME/.m2 - $HOME/flink_cache + # keep in sync with tools/travis/setup_maven.sh + - $HOME/maven_cache # do not cache our own artifacts before_cache: @@ -51,14 +53,7 @@ before_script: # Install maven 3.2.5 since trusty uses 3.3.9 for which shading is broken before_install: - - "wget https://archive.apache.org/dist/maven/maven-3/3.2.5/binaries/apache-maven-3.2.5-bin.zip" - - "unzip -qq apache-maven-3.2.5-bin.zip" - - "rm apache-maven-3.2.5-bin.zip" - - "export M2_HOME=$PWD/apache-maven-3.2.5" - - "export PATH=$M2_HOME/bin:$PATH" - - "export MAVEN_OPTS=\"-Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS\"" -# just in case: clean up the .m2 home and remove invalid jar files - - 'test ! -d $HOME/.m2/repository/ || find $HOME/.m2/repository/ -name "*.jar" -exec sh -c ''if ! zip -T {} >/dev/null ; then echo "deleting invalid file: {}"; rm -f {} ; fi'' \;' + - source ./tools/travis/setup_maven.sh # Installing the specified docker compose version - sudo rm /usr/local/bin/docker-compose - curl -L https://github.com/docker/compose/releases/download/${DOCKER_COMPOSE_VERSION}/docker-compose-`uname -s`-`uname -m` > docker-compose diff --git a/tools/travis/setup_maven.sh b/tools/travis/setup_maven.sh new file mode 100755 index 0000000000000..a02457da44af1 --- /dev/null +++ b/tools/travis/setup_maven.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +MAVEN_VERSION="3.2.5" +MAVEN_CACHE_DIR=${HOME}/maven_cache +MAVEN_VERSIONED_DIR=${MAVEN_CACHE_DIR}/apache-maven-${MAVEN_VERSION} + +if [ ! -d "${MAVEN_VERSIONED_DIR}" ]; then + wget https://archive.apache.org/dist/maven/maven-3/${MAVEN_VERSION}/binaries/apache-maven-${MAVEN_VERSION}-bin.zip + unzip -d "${MAVEN_CACHE_DIR}" -qq "apache-maven-${MAVEN_VERSION}-bin.zip" + rm "/apache-maven-${MAVEN_VERSION}-bin.zip" +fi + +export M2_HOME="${MAVEN_VERSIONED_DIR}" +export PATH=${M2_HOME}/bin:${PATH} +export MAVEN_OPTS="-Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS" + +# just in case: clean up the .m2 home and remove invalid jar files +if [ -d "${HOME}/.m2/repository/" ]; then + find ${HOME}/.m2/repository/ -name "*.jar" -exec sh -c 'if ! zip -T {} >/dev/null ; then echo "deleting invalid file: {}"; rm -f {} ; fi' \; +fi + diff --git a/tools/travis_controller.sh b/tools/travis_controller.sh index 6aa08e0d659b8..4868057c6cfd0 100755 --- a/tools/travis_controller.sh +++ b/tools/travis_controller.sh @@ -17,6 +17,12 @@ # limitations under the License. ################################################################################ +echo $M2_HOME +echo $PATH +echo $MAVEN_OPTS + +mvn -version + CACHE_DIR="$HOME/flink_cache" CACHE_BUILD_DIR="$CACHE_DIR/$TRAVIS_BUILD_NUMBER" CACHE_FLINK_DIR="$CACHE_BUILD_DIR/flink" From 90043718f5ac5e859d77cf984d0be7cf1a36de46 Mon Sep 17 00:00:00 2001 From: lining Date: Fri, 15 Feb 2019 20:51:22 +0800 Subject: [PATCH 315/359] [FLINK-11424][metrics] Properly remove string/failing gauges --- .../flink/metrics/datadog/DatadogHttpReporter.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java index b7e1c24fa6034..5462b32354900 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java +++ b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java @@ -118,6 +118,7 @@ public void close() { public void report() { DatadogHttpRequest request = new DatadogHttpRequest(); + List gaugesToRemove = new ArrayList<>(); for (Map.Entry entry : gauges.entrySet()) { DGauge g = entry.getValue(); try { @@ -125,11 +126,19 @@ public void report() { // Flink uses Gauge to store many types other than Number g.getMetricValue(); request.addGauge(g); + } catch (ClassCastException e) { + LOGGER.info("The metric {} will not be reported because only number types are supported by this reporter.", g.getMetric()); + gaugesToRemove.add(entry.getKey()); } catch (Exception e) { - // Remove that Gauge if it's not of Number type - gauges.remove(entry.getKey()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("The metric {} will not be reported because it threw an exception.", g.getMetric(), e); + } else { + LOGGER.info("The metric {} will not be reported because it threw an exception.", g.getMetric()); + } + gaugesToRemove.add(entry.getKey()); } } + gaugesToRemove.forEach(gauges::remove); for (DCounter c : counters.values()) { request.addCounter(c); From b4366d44961b33afe6d092376ab32488987043e8 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Sat, 16 Feb 2019 09:41:12 +0100 Subject: [PATCH 316/359] [hotfix][travis] Remove stray slash --- tools/travis/setup_maven.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/travis/setup_maven.sh b/tools/travis/setup_maven.sh index a02457da44af1..1de4bd90ae970 100755 --- a/tools/travis/setup_maven.sh +++ b/tools/travis/setup_maven.sh @@ -24,7 +24,7 @@ MAVEN_VERSIONED_DIR=${MAVEN_CACHE_DIR}/apache-maven-${MAVEN_VERSION} if [ ! -d "${MAVEN_VERSIONED_DIR}" ]; then wget https://archive.apache.org/dist/maven/maven-3/${MAVEN_VERSION}/binaries/apache-maven-${MAVEN_VERSION}-bin.zip unzip -d "${MAVEN_CACHE_DIR}" -qq "apache-maven-${MAVEN_VERSION}-bin.zip" - rm "/apache-maven-${MAVEN_VERSION}-bin.zip" + rm "apache-maven-${MAVEN_VERSION}-bin.zip" fi export M2_HOME="${MAVEN_VERSIONED_DIR}" From bf23cc48eb8385837fa75f32e994edf855262184 Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 15 Jan 2019 17:12:03 +0800 Subject: [PATCH 317/359] [FLINK-10964][sql-client] SQL Client throws exception when paging through finished batch query This closes #7265. --- .../apache/flink/table/client/SqlClient.java | 27 +- .../client/cli/CliChangelogResultView.java | 2 +- .../flink/table/client/cli/CliClient.java | 54 +++- .../flink/table/client/cli/CliResultView.java | 23 +- .../table/client/cli/CliTableResultView.java | 2 +- .../flink/table/client/cli/CliClientTest.java | 59 +++-- .../table/client/cli/CliResultViewTest.java | 241 ++++++++++++++++++ .../table/client/cli/utils/TerminalUtils.java | 60 +++++ 8 files changed, 412 insertions(+), 56 deletions(-) create mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java create mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClient.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClient.java index 757099b4d7140..0a7a0afc082eb 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClient.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClient.java @@ -115,16 +115,23 @@ private void start() { * @param executor executor */ private void openCli(SessionContext context, Executor executor) { - final CliClient cli = new CliClient(context, executor); - // interactive CLI mode - if (options.getUpdateStatement() == null) { - cli.open(); - } - // execute single update statement - else { - final boolean success = cli.submitUpdate(options.getUpdateStatement()); - if (!success) { - throw new SqlClientException("Could not submit given SQL update statement to cluster."); + CliClient cli = null; + try { + cli = new CliClient(context, executor); + // interactive CLI mode + if (options.getUpdateStatement() == null) { + cli.open(); + } + // execute single update statement + else { + final boolean success = cli.submitUpdate(options.getUpdateStatement()); + if (!success) { + throw new SqlClientException("Could not submit given SQL update statement to cluster."); + } + } + } finally { + if (cli != null) { + cli.close(); } } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliChangelogResultView.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliChangelogResultView.java index 5388a3e89f09f..71bfe16689baf 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliChangelogResultView.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliChangelogResultView.java @@ -119,7 +119,7 @@ protected void refresh() { break; // stop retrieval if job is done case EOS: - stopRetrieval(); + stopRetrieval(false); break; default: List> changes = result.getPayload(); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java index 987032efd27aa..a7c98938460c2 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java @@ -18,6 +18,7 @@ package org.apache.flink.table.client.cli; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.client.SqlClientException; import org.apache.flink.table.client.cli.SqlCommandParser.SqlCommandCall; @@ -77,21 +78,19 @@ public class CliClient { private static final int SOURCE_MAX_SIZE = 50_000; - public CliClient(SessionContext context, Executor executor) { + /** + * Creates a CLI instance with a custom terminal. Make sure to close the CLI instance + * afterwards using {@link #close()}. + */ + @VisibleForTesting + public CliClient(Terminal terminal, SessionContext context, Executor executor) { + this.terminal = terminal; this.context = context; this.executor = executor; - try { - // initialize terminal - terminal = TerminalBuilder.builder() - .name(CliStrings.CLI_NAME) - .build(); - // make space from previous output and test the writer - terminal.writer().println(); - terminal.writer().flush(); - } catch (IOException e) { - throw new SqlClientException("Error opening command line interface.", e); - } + // make space from previous output and test the writer + terminal.writer().println(); + terminal.writer().flush(); // initialize line lineReader lineReader = LineReaderBuilder.builder() @@ -117,6 +116,14 @@ public CliClient(SessionContext context, Executor executor) { .toAnsi(); } + /** + * Creates a CLI instance with a prepared terminal. Make sure to close the CLI instance + * afterwards using {@link #close()}. + */ + public CliClient(SessionContext context, Executor executor) { + this(createDefaultTerminal(), context, executor); + } + public Terminal getTerminal() { return terminal; } @@ -194,6 +201,17 @@ public void open() { } } + /** + * Closes the CLI instance. + */ + public void close() { + try { + terminal.close(); + } catch (IOException e) { + throw new SqlClientException("Unable to close terminal.", e); + } + } + /** * Submits a SQL update statement and prints status information and/or errors on the terminal. * @@ -530,4 +548,16 @@ private void printInfo(String message) { terminal.writer().println(CliStrings.messageInfo(message).toAnsi()); terminal.flush(); } + + // -------------------------------------------------------------------------------------------- + + private static Terminal createDefaultTerminal() { + try { + return TerminalBuilder.builder() + .name(CliStrings.CLI_NAME) + .build(); + } catch (IOException e) { + throw new SqlClientException("Error opening command line interface.", e); + } + } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java index 1fdd005cd6375..67ce5ebb21141 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliResultView.java @@ -150,8 +150,9 @@ protected void openRow() { view.open(); // enter view } - protected void stopRetrieval() { + protected void stopRetrieval(boolean cleanUpQuery) { // stop retrieval + refreshThread.cleanUpQuery = cleanUpQuery; refreshThread.isRunning = false; synchronized (refreshThread) { refreshThread.notify(); @@ -217,7 +218,7 @@ protected List computeMainLines() { @Override protected void cleanUp() { - stopRetrieval(); + stopRetrieval(true); } // -------------------------------------------------------------------------------------------- @@ -226,6 +227,8 @@ private class RefreshThread extends Thread { public volatile boolean isRunning = true; + public volatile boolean cleanUpQuery = true; + public long lastUpdatedResults = System.currentTimeMillis(); @Override @@ -278,13 +281,15 @@ public void run() { } } - // cancel table program - try { - // the cancellation happens in the refresh thread in order to keep the main thread - // responsive at all times; esp. if the cluster is not available - client.getExecutor().cancelQuery(client.getContext(), resultDescriptor.getResultId()); - } catch (SqlExecutionException e) { - // ignore further exceptions + if (cleanUpQuery) { + // cancel table program + try { + // the cancellation happens in the refresh thread in order to keep the main thread + // responsive at all times; esp. if the cluster is not available + client.getExecutor().cancelQuery(client.getContext(), resultDescriptor.getResultId()); + } catch (SqlExecutionException e) { + // ignore further exceptions + } } } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableResultView.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableResultView.java index 9c218f6d89ead..9a8cbcc9a8e7c 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableResultView.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableResultView.java @@ -96,7 +96,7 @@ protected void refresh() { // stop retrieval if job is done if (result.getType() == TypedResult.ResultType.EOS) { - stopRetrieval(); + stopRetrieval(false); } // update page else if (result.getType() == TypedResult.ResultType.PAYLOAD) { diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java index 4bf284b3ccb0e..2f8f2b59cc0e3 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.client.cli.utils.TerminalUtils; import org.apache.flink.table.client.config.Environment; import org.apache.flink.table.client.gateway.Executor; import org.apache.flink.table.client.gateway.ProgramTargetDescriptor; @@ -35,8 +36,10 @@ import org.jline.reader.LineReaderBuilder; import org.jline.reader.ParsedLine; import org.jline.reader.Parser; +import org.jline.terminal.Terminal; import org.junit.Test; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -70,7 +73,7 @@ public void testFailedUpdateSubmission() { } @Test - public void testSqlCompletion() { + public void testSqlCompletion() throws IOException { verifySqlCompletion("", 0, Arrays.asList("SELECT", "QUIT;", "RESET;"), Collections.emptyList()); verifySqlCompletion("SELEC", 5, Collections.singletonList("SELECT"), Collections.singletonList("QUIT;")); verifySqlCompletion("SELE", 0, Collections.singletonList("SELECT"), Collections.singletonList("QUIT;")); @@ -88,41 +91,51 @@ private void verifyUpdateSubmission(String statement, boolean failExecution, boo final MockExecutor mockExecutor = new MockExecutor(); mockExecutor.failExecution = failExecution; - final CliClient client = new CliClient(context, mockExecutor); - if (testFailure) { - assertFalse(client.submitUpdate(statement)); - } else { - assertTrue(client.submitUpdate(statement)); - assertEquals(statement, mockExecutor.receivedStatement); - assertEquals(context, mockExecutor.receivedContext); + CliClient cli = null; + try { + cli = new CliClient(TerminalUtils.createDummyTerminal(), context, mockExecutor); + if (testFailure) { + assertFalse(cli.submitUpdate(statement)); + } else { + assertTrue(cli.submitUpdate(statement)); + assertEquals(statement, mockExecutor.receivedStatement); + assertEquals(context, mockExecutor.receivedContext); + } + } finally { + if (cli != null) { + cli.close(); + } } } - private void verifySqlCompletion(String statement, int position, List expectedHints, List notExpectedHints) { + private void verifySqlCompletion(String statement, int position, List expectedHints, List notExpectedHints) throws IOException { final SessionContext context = new SessionContext("test-session", new Environment()); final MockExecutor mockExecutor = new MockExecutor(); final SqlCompleter completer = new SqlCompleter(context, mockExecutor); final SqlMultiLineParser parser = new SqlMultiLineParser(); - final LineReader reader = LineReaderBuilder.builder().build(); - final ParsedLine parsedLine = parser.parse(statement, position, Parser.ParseContext.COMPLETE); - final List candidates = new ArrayList<>(); - final List results = new ArrayList<>(); - completer.complete(reader, parsedLine, candidates); - candidates.forEach(item -> results.add(item.value())); + try (Terminal terminal = TerminalUtils.createDummyTerminal()) { + final LineReader reader = LineReaderBuilder.builder().terminal(terminal).build(); - assertTrue(results.containsAll(expectedHints)); + final ParsedLine parsedLine = parser.parse(statement, position, Parser.ParseContext.COMPLETE); + final List candidates = new ArrayList<>(); + final List results = new ArrayList<>(); + completer.complete(reader, parsedLine, candidates); + candidates.forEach(item -> results.add(item.value())); - assertEquals(statement, mockExecutor.receivedStatement); - assertEquals(context, mockExecutor.receivedContext); - assertEquals(position, mockExecutor.receivedPosition); - assertTrue(results.contains("HintA")); - assertTrue(results.contains("Hint B")); + assertTrue(results.containsAll(expectedHints)); - results.retainAll(notExpectedHints); - assertEquals(0, results.size()); + assertEquals(statement, mockExecutor.receivedStatement); + assertEquals(context, mockExecutor.receivedContext); + assertEquals(position, mockExecutor.receivedPosition); + assertTrue(results.contains("HintA")); + assertTrue(results.contains("Hint B")); + + results.retainAll(notExpectedHints); + assertEquals(0, results.size()); + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java new file mode 100644 index 0000000000000..0d090bd3f8766 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java @@ -0,0 +1,241 @@ +/* + * 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.table.client.cli; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.client.cli.utils.TerminalUtils; +import org.apache.flink.table.client.config.Environment; +import org.apache.flink.table.client.gateway.Executor; +import org.apache.flink.table.client.gateway.ProgramTargetDescriptor; +import org.apache.flink.table.client.gateway.ResultDescriptor; +import org.apache.flink.table.client.gateway.SessionContext; +import org.apache.flink.table.client.gateway.SqlExecutionException; +import org.apache.flink.table.client.gateway.TypedResult; +import org.apache.flink.types.Row; + +import org.jline.utils.AttributedString; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertTrue; + +/** + * Contains basic tests for the {@link CliResultView}. + */ +public class CliResultViewTest { + + @Test + public void testTableResultViewKeepJobResult() throws Exception { + testResultViewClearResult(TypedResult.endOfStream(), true, 0); + } + + @Test + public void testTableResultViewClearEmptyResult() throws Exception { + testResultViewClearResult(TypedResult.empty(), true, 1); + } + + @Test + public void testTableResultViewClearPayloadResult() throws Exception { + testResultViewClearResult(TypedResult.payload(1), true, 1); + } + + @Test + public void testChangelogResultViewKeepJobResult() throws Exception { + testResultViewClearResult(TypedResult.endOfStream(), false, 0); + } + + @Test + public void testChangelogResultViewClearEmptyResult() throws Exception { + testResultViewClearResult(TypedResult.empty(), false, 1); + } + + @Test + public void testChangelogResultViewClearPayloadResult() throws Exception { + testResultViewClearResult(TypedResult.payload(Collections.emptyList()), false, 1); + } + + private void testResultViewClearResult(TypedResult typedResult, boolean isTableMode, int expectedCancellationCount) throws Exception { + final CountDownLatch cancellationCounterLatch = new CountDownLatch(expectedCancellationCount); + final SessionContext session = new SessionContext("test-session", new Environment()); + final MockExecutor executor = new MockExecutor(typedResult, cancellationCounterLatch); + final ResultDescriptor descriptor = new ResultDescriptor( + "result-id", + TableSchema.builder().field("Null Field", Types.STRING()).build(), + false); + + Thread resultViewRunner = null; + CliClient cli = null; + try { + cli = new CliClient(TerminalUtils.createDummyTerminal(), session, executor); + resultViewRunner = new Thread(new TestingCliResultView(cli, descriptor, isTableMode)); + resultViewRunner.start(); + } finally { + if (resultViewRunner != null && !resultViewRunner.isInterrupted()) { + resultViewRunner.interrupt(); + } + if (cli != null) { + cli.close(); + } + } + + assertTrue( + "Invalid number of cancellations.", + cancellationCounterLatch.await(10, TimeUnit.SECONDS)); + } + + private static final class MockExecutor implements Executor { + + private final TypedResult typedResult; + private final CountDownLatch cancellationCounter; + + public MockExecutor(TypedResult typedResult, CountDownLatch cancellationCounter) { + this.typedResult = typedResult; + this.cancellationCounter = cancellationCounter; + } + + @Override + public void start() throws SqlExecutionException { + // do nothing + } + + @Override + public Map getSessionProperties(SessionContext session) throws SqlExecutionException { + return null; + } + + @Override + public List listTables(SessionContext session) throws SqlExecutionException { + return null; + } + + @Override + public List listUserDefinedFunctions(SessionContext session) throws SqlExecutionException { + return null; + } + + @Override + public TableSchema getTableSchema(SessionContext session, String name) throws SqlExecutionException { + return null; + } + + @Override + public String explainStatement(SessionContext session, String statement) throws SqlExecutionException { + return null; + } + + @Override + public List completeStatement(SessionContext session, String statement, int position) { + return null; + } + + @Override + public ResultDescriptor executeQuery(SessionContext session, String query) throws SqlExecutionException { + return null; + } + + @Override + @SuppressWarnings("unchecked") + public TypedResult>> retrieveResultChanges(SessionContext session, String resultId) throws SqlExecutionException { + return (TypedResult>>) typedResult; + } + + @Override + @SuppressWarnings("unchecked") + public TypedResult snapshotResult(SessionContext session, String resultId, int pageSize) throws SqlExecutionException { + return (TypedResult) typedResult; + } + + @Override + public List retrieveResultPage(String resultId, int page) throws SqlExecutionException { + return Collections.singletonList(new Row(1)); + } + + @Override + public void cancelQuery(SessionContext session, String resultId) throws SqlExecutionException { + cancellationCounter.countDown(); + } + + @Override + public ProgramTargetDescriptor executeUpdate(SessionContext session, String statement) throws SqlExecutionException { + return null; + } + + @Override + public void validateSession(SessionContext session) throws SqlExecutionException { + // do nothing + } + + @Override + public void stop(SessionContext session) { + // do nothing + } + } + + private static final class TestingCliResultView implements Runnable { + + private final CliResultView realResultView; + + public TestingCliResultView( + CliClient client, + ResultDescriptor descriptor, + boolean isTableMode) { + + if (isTableMode) { + realResultView = new TestingCliTableResultView(client, descriptor); + } else { + realResultView = new TestingCliChangelogResultView(client, descriptor); + } + } + + @Override + public void run() { + realResultView.open(); + } + } + + private static class TestingCliChangelogResultView extends CliChangelogResultView { + + public TestingCliChangelogResultView(CliClient client, ResultDescriptor resultDescriptor) { + super(client, resultDescriptor); + } + + @Override + protected List computeMainHeaderLines() { + return Collections.emptyList(); + } + } + + private static class TestingCliTableResultView extends CliTableResultView { + + public TestingCliTableResultView(CliClient client, ResultDescriptor resultDescriptor) { + super(client, resultDescriptor); + } + + @Override + protected List computeMainHeaderLines() { + return Collections.emptyList(); + } + } +} diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java new file mode 100644 index 0000000000000..772fb013e3c3c --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java @@ -0,0 +1,60 @@ +/* + * 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.table.client.cli.utils; + +import org.jline.terminal.Terminal; +import org.jline.terminal.impl.DumbTerminal; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * Utilities for terminal handling. + */ +public class TerminalUtils { + + private TerminalUtils() { + // do not instantiate + } + + public static Terminal createDummyTerminal() { + try { + return new DumbTerminal(new MockInputStream(), new MockOutputStream()); + } catch (IOException e) { + throw new RuntimeException("Unable to create dummy terminal."); + } + } + + private static class MockInputStream extends InputStream { + + @Override + public int read() { + return 0; + } + } + + private static class MockOutputStream extends OutputStream { + + @Override + public void write(int b) { + // do nothing + } + } +} From b9f78bbe68889f9398642bca753ce74638c0d4e6 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 19 Feb 2019 12:39:16 +0100 Subject: [PATCH 318/359] [FLINK-11041][tests] Fix ReinterpretDataStreamAsKeyedStreamITCase --- ...nterpretDataStreamAsKeyedStreamITCase.java | 73 +++++++++++-------- 1 file changed, 41 insertions(+), 32 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java index ff28dffff1d48..3bf1573d6e420 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java @@ -46,7 +46,6 @@ import java.io.BufferedOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import java.io.EOFException; import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; @@ -71,10 +70,11 @@ public class ReinterpretDataStreamAsKeyedStreamITCase { @Test public void testReinterpretAsKeyedStream() throws Exception { - final int numEventsPerInstance = 100; final int maxParallelism = 8; + final int numEventsPerInstance = 100; final int parallelism = 3; - final int numUniqueKeys = 12; + final int numTotalEvents = numEventsPerInstance * parallelism; + final int numUniqueKeys = 100; final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); @@ -102,7 +102,7 @@ public void testReinterpretAsKeyedStream() throws Exception { .timeWindow(Time.seconds(1)) // test that also timers and aggregated state work as expected .reduce((ReduceFunction>) (value1, value2) -> new Tuple2<>(value1.f0, value1.f1 + value2.f1)) - .addSink(new ValidatingSink(numEventsPerInstance * parallelism)).setParallelism(1); + .addSink(new ValidatingSink(numTotalEvents)).setParallelism(1); env.execute(); } @@ -110,10 +110,10 @@ public void testReinterpretAsKeyedStream() throws Exception { private static class RandomTupleSource implements ParallelSourceFunction> { private static final long serialVersionUID = 1L; - private int numKeys; + private final int numKeys; private int remainingEvents; - public RandomTupleSource(int numEvents, int numKeys) { + RandomTupleSource(int numEvents, int numKeys) { this.numKeys = numKeys; this.remainingEvents = numEvents; } @@ -121,9 +121,10 @@ public RandomTupleSource(int numEvents, int numKeys) { @Override public void run(SourceContext> out) throws Exception { Random random = new Random(42); - while (--remainingEvents >= 0) { + while (remainingEvents > 0) { synchronized (out.getCheckpointLock()) { out.collect(new Tuple2<>(random.nextInt(numKeys), 1)); + --remainingEvents; } } } @@ -141,7 +142,7 @@ private static class ToPartitionFileSink extends RichSinkFunction allPartitions; private DataOutputStream dos; - public ToPartitionFileSink(List allPartitions) { + ToPartitionFileSink(List allPartitions) { this.allPartitions = allPartitions; } @@ -171,17 +172,19 @@ private static class FromPartitionFileSource extends RichParallelSourceFunction< implements CheckpointedFunction, CheckpointListener { private static final long serialVersionUID = 1L; - private List allPartitions; + private final List allPartitions; private DataInputStream din; private volatile boolean running; + private long fileLength; + private long waitForFailurePos; private long position; private transient ListState positionState; private transient boolean isRestored; private transient volatile boolean canFail; - public FromPartitionFileSource(List allPartitions) { + FromPartitionFileSource(List allPartitions) { this.allPartitions = allPartitions; } @@ -189,9 +192,12 @@ public FromPartitionFileSource(List allPartitions) { public void open(Configuration parameters) throws Exception { super.open(parameters); int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + File partitionFile = allPartitions.get(subtaskIdx); + fileLength = partitionFile.length(); + waitForFailurePos = fileLength * 3 / 4; din = new DataInputStream( new BufferedInputStream( - new FileInputStream(allPartitions.get(subtaskIdx)))); + new FileInputStream(partitionFile))); long toSkip = position; while (toSkip > 0L) { @@ -207,33 +213,35 @@ public void close() throws Exception { @Override public void run(SourceContext> out) throws Exception { - this.running = true; - try { - while (running) { - checkFail(); + running = true; - synchronized (out.getCheckpointLock()) { - Integer key = din.readInt(); - Integer val = din.readInt(); - out.collect(new Tuple2<>(key, val)); + while (running && hasMoreDataToRead()) { - position += 2 * Integer.BYTES; - } + synchronized (out.getCheckpointLock()) { + Integer key = din.readInt(); + Integer val = din.readInt(); + out.collect(new Tuple2<>(key, val)); + + position += 2 * Integer.BYTES; } - } catch (EOFException ignore) { - if (!isRestored) { - while (true) { - checkFail(); + + if (shouldWaitForCompletedCheckpointAndFailNow()) { + while (!canFail) { + // wait for a checkpoint to complete + Thread.sleep(10L); } + throw new Exception("Artificial failure."); } } } - private void checkFail() throws Exception { - if (canFail) { - throw new Exception("Artificial failure."); - } + private boolean shouldWaitForCompletedCheckpointAndFailNow() { + return !isRestored && position > waitForFailurePos; + } + + private boolean hasMoreDataToRead() { + return position < fileLength; } @Override @@ -248,6 +256,7 @@ public void notifyCheckpointComplete(long checkpointId) { @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { + positionState.clear(); positionState.add(position); } @@ -260,8 +269,7 @@ public void initializeState(FunctionInitializationContext context) throws Except new ListStateDescriptor<>("posState", Long.class)); if (isRestored) { - - for (Long value : positionState.get()) { + for (long value : positionState.get()) { position += value; } } @@ -300,6 +308,7 @@ public void close() throws Exception { @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { + sumState.clear(); sumState.add(runningSum); } @@ -309,7 +318,7 @@ public void initializeState(FunctionInitializationContext context) throws Except new ListStateDescriptor<>("sumState", Integer.class)); if (context.isRestored()) { - for (Integer value : sumState.get()) { + for (int value : sumState.get()) { runningSum += value; } } From b0c7fe85a3168ecd156d69e486f9f4edc08aee16 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 20 Feb 2019 15:04:54 +0100 Subject: [PATCH 319/359] [FLINK-11690][runtime] Use configured RPC timeout in MiniCluster --- .../java/org/apache/flink/runtime/minicluster/MiniCluster.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 69bad47e15813..758a9cd8c20ba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -188,7 +188,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { public MiniCluster(MiniClusterConfiguration miniClusterConfiguration) { this.miniClusterConfiguration = checkNotNull(miniClusterConfiguration, "config may not be null"); - this.rpcTimeout = Time.seconds(10L); + this.rpcTimeout = miniClusterConfiguration.getRpcTimeout(); this.terminationFuture = CompletableFuture.completedFuture(null); running = false; } From 60144df3f75505625e04ba874bb6c87c0c02eec6 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 21 Feb 2019 11:23:03 +0100 Subject: [PATCH 320/359] [FLINK-11713][docs] Remove legacy mode from documentation --- docs/_includes/generated/mesos_configuration.html | 10 ---------- docs/monitoring/rest_api.md | 2 -- docs/ops/deployment/mesos.md | 6 ------ docs/ops/state/large_state_tuning.md | 2 -- .../apache/flink/mesos/configuration/MesosOptions.java | 2 ++ 5 files changed, 2 insertions(+), 20 deletions(-) diff --git a/docs/_includes/generated/mesos_configuration.html b/docs/_includes/generated/mesos_configuration.html index 54e92e5680c51..9d5be9f358026 100644 --- a/docs/_includes/generated/mesos_configuration.html +++ b/docs/_includes/generated/mesos_configuration.html @@ -12,21 +12,11 @@ 604800 The failover timeout in seconds for the Mesos scheduler, after which running tasks are automatically shut down. - -
    mesos.initial-tasks
    - 0 - The initial workers to bring up when the master starts. This option is ignored unless Flink is in
    legacy mode. -
    mesos.master
    (none) The Mesos master URL. The value should be in one of the following forms:
    • host:port
    • zk://host1:port1,host2:port2,.../path
    • zk://username:password@host1:port1,host2:port2,.../path
    • file:///path/to/file
    - -
    mesos.maximum-failed-tasks
    - -1 - The maximum number of failed workers before the cluster fails. May be set to -1 to disable this feature. This option is ignored unless Flink is in legacy mode. -
    mesos.resourcemanager.artifactserver.port
    0 diff --git a/docs/monitoring/rest_api.md b/docs/monitoring/rest_api.md index c799e915734f1..f433163758112 100644 --- a/docs/monitoring/rest_api.md +++ b/docs/monitoring/rest_api.md @@ -61,8 +61,6 @@ If no version is specified Flink will default to the *oldest* version supporting Querying unsupported/non-existing versions will return a 404 error. -Attention REST API versioning is *not* active if the cluster runs in [legacy mode](../ops/config.html#mode). For this case please refer to the legacy API below. -
    diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md index 1ff8afad74ebb..b36bdec896b35 100644 --- a/docs/ops/deployment/mesos.md +++ b/docs/ops/deployment/mesos.md @@ -189,12 +189,6 @@ For example: -Dtaskmanager.numberOfTaskSlots=2 \ -Dparallelism.default=10 -
    - Note: If Flink is in legacy mode, - you should additionally define the number of task managers that are started by Mesos via - mesos.initial-tasks. -
    - ### High Availability You will need to run a service like Marathon or Apache Aurora which takes care of restarting the Flink master process in case of node or process failures. diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md index 47154af58a4fa..61956e5471453 100644 --- a/docs/ops/state/large_state_tuning.md +++ b/docs/ops/state/large_state_tuning.md @@ -332,6 +332,4 @@ if a task manager is no longer available, a task that cannot return to its previ that the previous slot can only disappear when a task manager is no longer available, and in this case *some* tasks have to request a new slot anyways. With our scheduling strategy we give the maximum number of tasks a chance to recover from their local state and avoid the cascading effect of tasks stealing their previous slots from one another. -Allocation-preserving scheduling does not work with Flink's legacy mode. - {% top %} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java b/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java index 0c4e1f6bcba4b..93f362573cc88 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/configuration/MesosOptions.java @@ -33,6 +33,7 @@ public class MesosOptions { /** * The initial number of Mesos tasks to allocate. */ + @Deprecated public static final ConfigOption INITIAL_TASKS = key("mesos.initial-tasks") .defaultValue(0) @@ -47,6 +48,7 @@ public class MesosOptions { * *

    By default, we take the number of initially requested tasks. */ + @Deprecated public static final ConfigOption MAX_FAILED_TASKS = key("mesos.maximum-failed-tasks") .defaultValue(-1) From 9185eec91674477e832601863500e105dcf47f5a Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Mon, 25 Feb 2019 14:24:06 +0100 Subject: [PATCH 321/359] [FLINK-11745][State TTL][E2E] Restore from the savepoint after the job cancellation. --- .../test-scripts/test_stream_state_ttl.sh | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh b/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh index fb911f327ff90..1a634260db630 100755 --- a/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh +++ b/flink-end-to-end-tests/test-scripts/test_stream_state_ttl.sh @@ -22,9 +22,8 @@ source "$(dirname "$0")"/common.sh STATE_BACKEND_TYPE="${1:-file}" STATE_BACKEND_FILE_ASYNC="${2:-false}" TTL="${3:-1000}" -PRECISION="${4:-5}" -PARALLELISM="${5-3}" -UPDATE_NUM="${6-1000}" +PARALLELISM="${4-3}" +UPDATE_NUM="${5-1000}" CHECKPOINT_DIR="file://$TEST_DATA_DIR/savepoint-e2e-test-chckpt-dir" @@ -46,15 +45,19 @@ trap test_cleanup INT trap test_cleanup EXIT start_cluster -start_taskmanagers $PARALLELISM +if [ "${PARALLELISM}" -gt "1" ]; then + start_taskmanagers $(expr ${PARALLELISM} - 1) +fi function job_id() { - CMD="${FLINK_DIR}/bin/flink run -d -p ${PARALLELISM} ${TEST_PROGRAM_JAR} \ + if [ -n "$1" ]; then + SP="-s $1" + fi + CMD="${FLINK_DIR}/bin/flink run -d ${SP} -p ${PARALLELISM} ${TEST_PROGRAM_JAR} \ --test.semantics exactly-once \ --environment.parallelism ${PARALLELISM} \ --state_backend ${STATE_BACKEND_TYPE} \ --state_ttl_verifier.ttl_milli ${TTL} \ - --state_ttl_verifier.precision_milli ${PRECISION} \ --state_backend.checkpoint_directory ${CHECKPOINT_DIR} \ --state_backend.file.async ${STATE_BACKEND_FILE_ASYNC} \ --update_generator_source.sleep_time 10 \ @@ -73,7 +76,7 @@ SAVEPOINT_PATH=$(take_savepoint ${JOB} ${TEST_DATA_DIR} \ cancel_job ${JOB} -JOB_CMD=$(job_id) +JOB_CMD=$(job_id ${SAVEPOINT_PATH}) echo ${JOB_CMD} JOB=$(${JOB_CMD} | grep 'Job has been submitted with JobID' | sed 's/.* //g') wait_job_running ${JOB} From 55a3ca77b6f70bc59c1919a44e53145704504003 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 26 Feb 2019 16:53:14 +0100 Subject: [PATCH 322/359] [FLINK-11185] Fix StreamSourceOperatorWatermarksTest instability. The cause of the instability seems to be that due to a not-so-rare timing, the thread that calls the `interrupt()` on the main thread, runs still after its original test finishes and calls `interrupt()` during execution of the next test. This causes the normal execution (or `sleep()` in this case) to be interrupted. --- .../runtime/operators/StreamSourceOperatorWatermarksTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java index 4b5259e91c41d..00c3ac4ee7141 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java @@ -101,7 +101,6 @@ public void testNoMaxWatermarkOnImmediateCancel() throws Exception { public void testNoMaxWatermarkOnAsyncCancel() throws Exception { final List output = new ArrayList<>(); - final Thread runner = Thread.currentThread(); // regular stream source operator final StreamSource> operator = @@ -117,7 +116,6 @@ public void run() { Thread.sleep(200); } catch (InterruptedException ignored) {} operator.cancel(); - runner.interrupt(); } }.start(); From 185693f71486d1a7f3d396c5b303de7f46e9420c Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Mon, 25 Feb 2019 15:11:55 +0100 Subject: [PATCH 323/359] [FLINK-10881] Use cancelWithSavepoint in SavepointITCase test. --- .../org/apache/flink/test/checkpointing/SavepointITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 9140570a89009..67cc9b7aec9e6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -192,7 +192,7 @@ private String submitJobAndTakeSavepoint(MiniClusterResourceFactory clusterFacto StatefulCounter.getProgressLatch().await(); - return client.triggerSavepoint(jobId, null).get(); + return client.cancelWithSavepoint(jobId, null); } finally { cluster.after(); StatefulCounter.resetForTest(parallelism); From 8b01a241d27afdceb3ff250d25ad31f63d982f75 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 28 Feb 2019 21:46:33 +0800 Subject: [PATCH 324/359] [hotfix] [docs] Bump latest stable version to 1.7.2 --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index bfa24aefa1683..3819e27463d01 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -27,7 +27,7 @@ # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) -version: "1.7.1" +version: "1.7.2" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version version_title: "1.7" From 837cc07450d38b0e1aff11d0b8d2d75ac9ac682f Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Thu, 28 Feb 2019 12:12:09 +0100 Subject: [PATCH 325/359] [FLINK-11787] Update Kubernetes resources: workaround to make TM reachable from JM in Kubernetes (for Flink 1.7 only) This closes #7858. --- docs/ops/deployment/kubernetes.md | 5 +++++ .../kubernetes/task-manager-deployment.yaml.template | 7 ++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/docs/ops/deployment/kubernetes.md b/docs/ops/deployment/kubernetes.md index 5244f5ed544b3..ef0a0358b8fe9 100644 --- a/docs/ops/deployment/kubernetes.md +++ b/docs/ops/deployment/kubernetes.md @@ -147,6 +147,7 @@ spec: image: flink:latest args: - taskmanager + - "-Dtaskmanager.host=$(K8S_POD_IP)" ports: - containerPort: 6121 name: data @@ -157,6 +158,10 @@ spec: env: - name: JOB_MANAGER_RPC_ADDRESS value: flink-jobmanager + - name: K8S_POD_IP + valueFrom: + fieldRef: + fieldPath: status.podIP {% endhighlight %} `jobmanager-service.yaml` diff --git a/flink-container/kubernetes/task-manager-deployment.yaml.template b/flink-container/kubernetes/task-manager-deployment.yaml.template index 55bd9a934c22f..f75b73234fa70 100644 --- a/flink-container/kubernetes/task-manager-deployment.yaml.template +++ b/flink-container/kubernetes/task-manager-deployment.yaml.template @@ -31,4 +31,9 @@ spec: containers: - name: flink-task-manager image: ${FLINK_IMAGE_NAME} - args: ["task-manager", "-Djobmanager.rpc.address=flink-job-cluster"] + args: ["task-manager", "-Djobmanager.rpc.address=flink-job-cluster", "-Dtaskmanager.host=$(K8S_POD_IP)"] + env: + - name: K8S_POD_IP + valueFrom: + fieldRef: + fieldPath: status.podIP From e3c0e69ce8920df15c1a85788891ff0e645e207b Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 26 Feb 2019 15:49:42 +0100 Subject: [PATCH 326/359] [FLINK-9003][E2E] Add operators with input type that goes through custom, stateful serialization This closes #7890. --- .../api/common/typeutils/TypeSerializer.java | 5 +- .../DataStreamAllroundTestJobFactory.java | 69 ++++++- .../tests/DataStreamAllroundTestProgram.java | 76 ++++--- .../streaming/tests/SemanticsCheckMapper.java | 2 +- .../tests/SequenceGeneratorSource.java | 7 +- .../SingleThreadAccessCheckingTypeInfo.java | 100 +++++++++ ...gleThreadAccessCheckingTypeSerializer.java | 195 ++++++++++++++++++ .../streaming/tests/TestOperatorEnum.java | 53 +++++ .../builder/ArtificialStateBuilder.java | 4 +- 9 files changed, 468 insertions(+), 43 deletions(-) create mode 100644 flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeInfo.java create mode 100644 flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeSerializer.java create mode 100644 flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/TestOperatorEnum.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java index ddb0b87e52574..5e9498ecf1ea8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java @@ -29,9 +29,8 @@ * This interface describes the methods that are required for a data type to be handled by the Flink * runtime. Specifically, this interface contains the serialization and copying methods. * - *

    The methods in this class are assumed to be stateless, such that it is effectively thread safe. Stateful - * implementations of the methods may lead to unpredictable side effects and will compromise both stability and - * correctness of the program. + *

    The methods in this class are not necessarily thread safe. To avoid unpredictable side effects, + * it is recommended to call {@code duplicate()} method and use one serializer instance per thread. * *

    Upgrading TypeSerializers to the new TypeSerializerSnapshot model * diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java index 4f69cdb4e0437..af39522aff96d 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java @@ -24,15 +24,20 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; +import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.KeyedStream; import org.apache.flink.streaming.api.datastream.WindowedStream; import org.apache.flink.streaming.api.environment.CheckpointConfig; @@ -51,6 +56,10 @@ import java.util.ArrayList; import java.util.List; +import static org.apache.flink.streaming.tests.TestOperatorEnum.EVENT_IDENTITY_MAPPER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.MAPPER_RETURNS_OUT_WITH_CUSTOM_SER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.RESULT_TYPE_QUERYABLE_MAPPER_WITH_CUSTOM_SER; + /** * A factory for components of general purpose test jobs for Flink's DataStream API operators and primitives. * @@ -268,13 +277,13 @@ public static void setupEnvironment(StreamExecutionEnvironment env, ParameterToo STATE_BACKEND_FILE_ASYNC.key(), STATE_BACKEND_FILE_ASYNC.defaultValue()); - env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints)); + env.setStateBackend((StateBackend) new FsStateBackend(checkpointDir, asyncCheckpoints)); } else if ("rocks".equalsIgnoreCase(stateBackend)) { boolean incrementalCheckpoints = pt.getBoolean( STATE_BACKEND_ROCKS_INCREMENTAL.key(), STATE_BACKEND_ROCKS_INCREMENTAL.defaultValue()); - env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints)); + env.setStateBackend((StateBackend) new RocksDBStateBackend(checkpointDir, incrementalCheckpoints)); } else { throw new IllegalArgumentException("Unknown backend requested: " + stateBackend); } @@ -441,7 +450,7 @@ static ArtificalOperatorStateMapper createArtificialOperatorS return new ArtificalOperatorStateMapper<>(mapFunction); } - static ArtificialStateBuilder createValueStateBuilder( + private static ArtificialStateBuilder createValueStateBuilder( JoinFunction inputAndOldStateToNewState, ValueStateDescriptor valueStateDescriptor) { @@ -451,7 +460,7 @@ static ArtificialStateBuilder createValueStateBuilder( valueStateDescriptor); } - static ArtificialStateBuilder createListStateBuilder( + private static ArtificialStateBuilder createListStateBuilder( JoinFunction inputAndOldStateToNewState, ListStateDescriptor listStateDescriptor) { @@ -489,4 +498,56 @@ static FlatMapFunction>, String> createSlidingWindow TEST_SLIDE_FACTOR.defaultValue() )); } + + static DataStream verifyCustomStatefulTypeSerializer(DataStream eventStream) { + return eventStream + .map(new EventIdentityFunctionWithCustomEventTypeInformation()) + .name(RESULT_TYPE_QUERYABLE_MAPPER_WITH_CUSTOM_SER.getName()) + .uid(RESULT_TYPE_QUERYABLE_MAPPER_WITH_CUSTOM_SER.getUid()) + // apply a keyBy so that we have a non-chained operator with Event as input type that goes through serialization + .keyBy(new EventKeySelectorWithCustomKeyTypeInformation()) + + .map(e -> e) + .returns(new SingleThreadAccessCheckingTypeInfo<>(Event.class)) + .name(MAPPER_RETURNS_OUT_WITH_CUSTOM_SER.getName()) + .uid(MAPPER_RETURNS_OUT_WITH_CUSTOM_SER.getUid()) + // apply a keyBy so that we have a non-chained operator with Event as input type that goes through serialization + .keyBy(new EventKeySelectorWithCustomKeyTypeInformation()) + + .map(e -> e) + .name(EVENT_IDENTITY_MAPPER.getName()) + .uid(EVENT_IDENTITY_MAPPER.getUid()); + } + + private static class EventIdentityFunctionWithCustomEventTypeInformation + implements MapFunction, ResultTypeQueryable { + + private final SingleThreadAccessCheckingTypeInfo typeInformation = new SingleThreadAccessCheckingTypeInfo<>(Event.class); + + @Override + public Event map(Event value) { + return value; + } + + @Override + public TypeInformation getProducedType() { + return typeInformation; + } + } + + private static class EventKeySelectorWithCustomKeyTypeInformation + implements KeySelector, ResultTypeQueryable { + + private final SingleThreadAccessCheckingTypeInfo typeInformation = new SingleThreadAccessCheckingTypeInfo<>(Integer.class); + + @Override + public Integer getKey(Event value) { + return value.getKey(); + } + + @Override + public TypeInformation getProducedType() { + return typeInformation; + } + } } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java index 1f2758ddc367b..3138af5de7363 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java @@ -52,6 +52,17 @@ import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createTimestampExtractor; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.isSimulateFailures; import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.setupEnvironment; +import static org.apache.flink.streaming.tests.TestOperatorEnum.EVENT_SOURCE; +import static org.apache.flink.streaming.tests.TestOperatorEnum.FAILURE_MAPPER_NAME; +import static org.apache.flink.streaming.tests.TestOperatorEnum.KEYED_STATE_OPER_WITH_AVRO_SER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.KEYED_STATE_OPER_WITH_KRYO_AND_CUSTOM_SER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.OPERATOR_STATE_OPER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.SEMANTICS_CHECK_MAPPER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.SEMANTICS_CHECK_PRINT_SINK; +import static org.apache.flink.streaming.tests.TestOperatorEnum.SLIDING_WINDOW_AGG; +import static org.apache.flink.streaming.tests.TestOperatorEnum.SLIDING_WINDOW_CHECK_MAPPER; +import static org.apache.flink.streaming.tests.TestOperatorEnum.SLIDING_WINDOW_CHECK_PRINT_SINK; +import static org.apache.flink.streaming.tests.TestOperatorEnum.TIME_WINDOW_OPER; /** * A general purpose test job for Flink's DataStream API operators and primitives. @@ -70,13 +81,6 @@ * */ public class DataStreamAllroundTestProgram { - private static final String KEYED_STATE_OPER_NAME = "ArtificalKeyedStateMapper"; - private static final String OPERATOR_STATE_OPER_NAME = "ArtificalOperatorStateMapper"; - private static final String TIME_WINDOW_OPER_NAME = "TumblingWindowOperator"; - private static final String SEMANTICS_CHECK_MAPPER_NAME = "SemanticsCheckMapper"; - private static final String FAILURE_MAPPER_NAME = "FailureMapper"; - private static final String SLIDING_WINDOW_CHECK_MAPPER_NAME = "SlidingWindowCheckMapper"; - private static final String SLIDING_WINDOW_AGG_NAME = "SlidingWindowOperator"; public static void main(String[] args) throws Exception { final ParameterTool pt = ParameterTool.fromArgs(args); @@ -86,7 +90,10 @@ public static void main(String[] args) throws Exception { setupEnvironment(env, pt); // add a keyed stateful map operator, which uses Kryo for state serialization - DataStream eventStream = env.addSource(createEventSource(pt)).uid("0001") + DataStream eventStream = env + .addSource(createEventSource(pt)) + .name(EVENT_SOURCE.getName()) + .uid(EVENT_SOURCE.getUid()) .assignTimestampsAndWatermarks(createTimestampExtractor(pt)) .keyBy(Event::getKey) .map(createArtificialKeyedStateMapper( @@ -94,18 +101,21 @@ public static void main(String[] args) throws Exception { (MapFunction) in -> in, // state is verified and updated per event as a wrapped ComplexPayload state object (Event event, ComplexPayload lastState) -> { - if (lastState != null && !lastState.getStrPayload().equals(KEYED_STATE_OPER_NAME) + if (lastState != null && !lastState.getStrPayload().equals(KEYED_STATE_OPER_WITH_KRYO_AND_CUSTOM_SER.getName()) && lastState.getInnerPayLoad().getSequenceNumber() == (event.getSequenceNumber() - 1)) { System.out.println("State is set or restored incorrectly"); } - return new ComplexPayload(event, KEYED_STATE_OPER_NAME); + return new ComplexPayload(event, KEYED_STATE_OPER_WITH_KRYO_AND_CUSTOM_SER.getName()); }, Arrays.asList( new KryoSerializer<>(ComplexPayload.class, env.getConfig()), // KryoSerializer new StatefulComplexPayloadSerializer()), // custom stateful serializer Collections.singletonList(ComplexPayload.class) // KryoSerializer via type extraction ) - ).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Kryo_and_Custom_Stateful").uid("0002"); + ) + .returns(Event.class) + .name(KEYED_STATE_OPER_WITH_KRYO_AND_CUSTOM_SER.getName()) + .uid(KEYED_STATE_OPER_WITH_KRYO_AND_CUSTOM_SER.getUid()); // add a keyed stateful map operator, which uses Avro for state serialization eventStream = eventStream @@ -115,7 +125,7 @@ public static void main(String[] args) throws Exception { (MapFunction) in -> in, // state is verified and updated per event as a wrapped ComplexPayloadAvro state object (Event event, ComplexPayloadAvro lastState) -> { - if (lastState != null && !lastState.getStrPayload().equals(KEYED_STATE_OPER_NAME) + if (lastState != null && !lastState.getStrPayload().equals(KEYED_STATE_OPER_WITH_AVRO_SER.getName()) && lastState.getInnerPayLoad().getSequenceNumber() == (event.getSequenceNumber() - 1)) { System.out.println("State is set or restored incorrectly"); } @@ -123,7 +133,7 @@ public static void main(String[] args) throws Exception { ComplexPayloadAvro payload = new ComplexPayloadAvro(); payload.setEventTime(event.getEventTime()); payload.setInnerPayLoad(new InnerPayLoadAvro(event.getSequenceNumber())); - payload.setStrPayload(KEYED_STATE_OPER_NAME); + payload.setStrPayload(KEYED_STATE_OPER_WITH_AVRO_SER.getName()); payload.setStringList(Arrays.asList(String.valueOf(event.getKey()), event.getPayload())); return payload; @@ -132,38 +142,49 @@ public static void main(String[] args) throws Exception { new AvroSerializer<>(ComplexPayloadAvro.class)), // custom AvroSerializer Collections.singletonList(ComplexPayloadAvro.class) // AvroSerializer via type extraction ) - ).returns(Event.class).name(KEYED_STATE_OPER_NAME + "_Avro").uid("0003"); + ) + .returns(Event.class) + .name(KEYED_STATE_OPER_WITH_AVRO_SER.getName()) + .uid(KEYED_STATE_OPER_WITH_AVRO_SER.getUid()); DataStream eventStream2 = eventStream .map(createArtificialOperatorStateMapper((MapFunction) in -> in)) .returns(Event.class) - .name(OPERATOR_STATE_OPER_NAME).uid("0004"); + .name(OPERATOR_STATE_OPER.getName()) + .uid(OPERATOR_STATE_OPER.getUid()); // apply a tumbling window that simply passes forward window elements; // this allows the job to cover timers state + @SuppressWarnings("Convert2Lambda") DataStream eventStream3 = applyTumblingWindows(eventStream2.keyBy(Event::getKey), pt) .apply(new WindowFunction() { @Override - public void apply(Integer integer, TimeWindow window, Iterable input, Collector out) throws Exception { + public void apply(Integer integer, TimeWindow window, Iterable input, Collector out) { for (Event e : input) { out.collect(e); } } - }).name(TIME_WINDOW_OPER_NAME).uid("0005"); + }) + .name(TIME_WINDOW_OPER.getName()) + .uid(TIME_WINDOW_OPER.getUid()); + + eventStream3 = DataStreamAllroundTestJobFactory.verifyCustomStatefulTypeSerializer(eventStream3); if (isSimulateFailures(pt)) { eventStream3 = eventStream3 .map(createFailureMapper(pt)) .setParallelism(1) - .name(FAILURE_MAPPER_NAME).uid("0006"); + .name(FAILURE_MAPPER_NAME.getName()) + .uid(FAILURE_MAPPER_NAME.getUid()); } eventStream3.keyBy(Event::getKey) .flatMap(createSemanticsCheckMapper(pt)) - .name(SEMANTICS_CHECK_MAPPER_NAME) - .uid("007") + .name(SEMANTICS_CHECK_MAPPER.getName()) + .uid(SEMANTICS_CHECK_MAPPER.getUid()) .addSink(new PrintSinkFunction<>()) - .uid("008"); + .name(SEMANTICS_CHECK_PRINT_SINK.getName()) + .uid(SEMANTICS_CHECK_PRINT_SINK.getUid()); // Check sliding windows aggregations. Output all elements assigned to a window and later on // check if each event was emitted slide_factor number of times @@ -175,20 +196,21 @@ public void apply(Integer integer, TimeWindow window, Iterable input, Col @Override public void apply( Integer key, TimeWindow window, Iterable input, - Collector>> out) throws Exception { + Collector>> out) { out.collect(Tuple2.of(key, StreamSupport.stream(input.spliterator(), false).collect(Collectors.toList()))); } }) - .name(SLIDING_WINDOW_AGG_NAME) - .uid("009"); + .name(SLIDING_WINDOW_AGG.getName()) + .uid(SLIDING_WINDOW_AGG.getUid()); eventStream4.keyBy(events-> events.f0) .flatMap(createSlidingWindowCheckMapper(pt)) - .uid("010") - .name(SLIDING_WINDOW_CHECK_MAPPER_NAME) + .name(SLIDING_WINDOW_CHECK_MAPPER.getName()) + .uid(SLIDING_WINDOW_CHECK_MAPPER.getUid()) .addSink(new PrintSinkFunction<>()) - .uid("011"); + .name(SLIDING_WINDOW_CHECK_PRINT_SINK.getName()) + .uid(SLIDING_WINDOW_CHECK_PRINT_SINK.getUid()); env.execute("General purpose test job"); } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java index 780e2aecd3e7b..1fe5f61ca7359 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java @@ -60,7 +60,7 @@ public void flatMap(Event event, Collector out) throws Exception { } @Override - public void open(Configuration parameters) throws Exception { + public void open(Configuration parameters) { ValueStateDescriptor sequenceStateDescriptor = new ValueStateDescriptor<>("sequenceState", Long.class); diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java index c6ecb3f768099..5dd09a9bfc4aa 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SequenceGeneratorSource.java @@ -28,9 +28,6 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -43,8 +40,6 @@ public class SequenceGeneratorSource extends RichParallelSourceFunction i private static final long serialVersionUID = -3986989644799442178L; - private static final Logger LOG = LoggerFactory.getLogger(SequenceGeneratorSource.class); - /** Length of the artificial payload string generated for each event. */ private final int payloadLength; @@ -145,7 +140,7 @@ private void runActive(SourceContext ctx) throws Exception { } } - private void runIdle(SourceContext ctx) throws Exception { + private void runIdle(SourceContext ctx) { ctx.markAsTemporarilyIdle(); // just wait until this source is canceled diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeInfo.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeInfo.java new file mode 100644 index 0000000000000..deb5b0c2893c4 --- /dev/null +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeInfo.java @@ -0,0 +1,100 @@ +/* + * 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.tests; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +import java.util.Objects; + +/** Custom {@link TypeInformation} to test custom {@link TypeSerializer}. */ +public class SingleThreadAccessCheckingTypeInfo extends TypeInformation { + private final TypeInformation originalTypeInformation; + + SingleThreadAccessCheckingTypeInfo(Class clazz) { + this(TypeInformation.of(clazz)); + } + + private SingleThreadAccessCheckingTypeInfo(TypeInformation originalTypeInformation) { + this.originalTypeInformation = originalTypeInformation; + } + + @Override + public boolean isBasicType() { + return originalTypeInformation.isBasicType(); + } + + @Override + public boolean isTupleType() { + return originalTypeInformation.isTupleType(); + } + + @Override + public int getArity() { + return originalTypeInformation.getArity(); + } + + @Override + public int getTotalFields() { + return originalTypeInformation.getTotalFields(); + } + + @Override + public Class getTypeClass() { + return originalTypeInformation.getTypeClass(); + } + + @Override + public boolean isKeyType() { + return originalTypeInformation.isKeyType(); + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return new SingleThreadAccessCheckingTypeSerializer<>(originalTypeInformation.createSerializer(config)); + } + + @Override + public String toString() { + return originalTypeInformation.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o){ + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SingleThreadAccessCheckingTypeInfo that = (SingleThreadAccessCheckingTypeInfo) o; + return Objects.equals(originalTypeInformation, that.originalTypeInformation); + } + + @Override + public int hashCode() { + return Objects.hash(originalTypeInformation); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof SingleThreadAccessCheckingTypeInfo; + } +} diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeSerializer.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeSerializer.java new file mode 100644 index 0000000000000..b162588dd222e --- /dev/null +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SingleThreadAccessCheckingTypeSerializer.java @@ -0,0 +1,195 @@ +/* + * 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.tests; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicReference; + +/** {@link TypeSerializer} to check mutually exclusive thread access API to methods. */ +@Internal +class SingleThreadAccessCheckingTypeSerializer extends TypeSerializer { + private static final long serialVersionUID = 131020282727167064L; + + private final SingleThreadAccessChecker singleThreadAccessChecker; + private final TypeSerializer originalSerializer; + + SingleThreadAccessCheckingTypeSerializer(TypeSerializer originalSerializer) { + this.singleThreadAccessChecker = new SingleThreadAccessChecker(); + this.originalSerializer = originalSerializer; + } + + @Override + public boolean isImmutableType() { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.isImmutableType(); + } + } + + @Override + public TypeSerializer duplicate() { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return new SingleThreadAccessCheckingTypeSerializer<>(originalSerializer.duplicate()); + } + } + + @Override + public T createInstance() { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.createInstance(); + } + } + + @Override + public T copy(T from) { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.copy(from); + } + } + + @Override + public T copy(T from, T reuse) { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.copy(from, reuse); + } + } + + @Override + public int getLength() { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.getLength(); + } + } + + @Override + public void serialize(T record, DataOutputView target) throws IOException { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + originalSerializer.serialize(record, target); + } + } + + @Override + public T deserialize(DataInputView source) throws IOException { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.deserialize(source); + } + } + + @Override + public T deserialize(T reuse, DataInputView source) throws IOException { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.deserialize(reuse, source); + } + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + originalSerializer.copy(source, target); + } + } + + @Override + public boolean equals(Object obj) { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return obj == this || + (obj != null && obj.getClass() == getClass() && + originalSerializer.equals(obj)); + } + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof SingleThreadAccessCheckingTypeSerializer; + } + + @Override + public int hashCode() { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return originalSerializer.hashCode(); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + return new SingleThreadAccessCheckingTypeSerializerSnapshot<>(this); + } + } + + /** {@link TypeSerializerSnapshot} for {@link SingleThreadAccessCheckingTypeSerializer}. */ + public static class SingleThreadAccessCheckingTypeSerializerSnapshot + extends CompositeTypeSerializerConfigSnapshot { + + SingleThreadAccessCheckingTypeSerializerSnapshot(SingleThreadAccessCheckingTypeSerializer serializerInstance) { + super(serializerInstance); + } + + @Override + public int getVersion() { + return 0; + } + } + + private void writeObject(ObjectOutputStream outputStream) throws IOException { + try (SingleThreadAccessCheck ignored = singleThreadAccessChecker.startSingleThreadAccessCheck()) { + outputStream.defaultWriteObject(); + } + } + + private static class SingleThreadAccessChecker implements Serializable { + private static final long serialVersionUID = 131020282727167064L; + + private transient AtomicReference currentThreadRef = new AtomicReference<>(); + + SingleThreadAccessCheck startSingleThreadAccessCheck() { + assert(currentThreadRef.compareAndSet(null, Thread.currentThread())) : + "The checker has concurrent access from " + currentThreadRef.get(); + return new SingleThreadAccessCheck(currentThreadRef); + } + + private void readObject(ObjectInputStream inputStream) throws ClassNotFoundException, IOException { + inputStream.defaultReadObject(); + currentThreadRef = new AtomicReference<>(); + } + } + + private static class SingleThreadAccessCheck implements AutoCloseable { + private final AtomicReference currentThreadRef; + + private SingleThreadAccessCheck(AtomicReference currentThreadRef) { + this.currentThreadRef = currentThreadRef; + } + + @Override + public void close() { + assert(currentThreadRef.compareAndSet(Thread.currentThread(), null)) : + "The checker has concurrent access from " + currentThreadRef.get(); + } + } +} diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/TestOperatorEnum.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/TestOperatorEnum.java new file mode 100644 index 0000000000000..4769cedd9ba5e --- /dev/null +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/TestOperatorEnum.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.tests; + +/** Enum of names and uids of all test operators used in {@link DataStreamAllroundTestProgram}. */ +public enum TestOperatorEnum { + EVENT_SOURCE("EventSource", 1), + KEYED_STATE_OPER_WITH_KRYO_AND_CUSTOM_SER("ArtificalKeyedStateMapper_Kryo_and_Custom_Stateful", 2), + KEYED_STATE_OPER_WITH_AVRO_SER("ArtificalKeyedStateMapper_Avro", 3), + OPERATOR_STATE_OPER("ArtificalOperatorStateMapper", 4), + TIME_WINDOW_OPER("TumblingWindowOperator", 5), + FAILURE_MAPPER_NAME("FailureMapper", 6), + SEMANTICS_CHECK_MAPPER("SemanticsCheckMapper", 7), + SEMANTICS_CHECK_PRINT_SINK("SemanticsCheckPrintSink", 8), + SLIDING_WINDOW_AGG("SlidingWindowOperator", 9), + SLIDING_WINDOW_CHECK_MAPPER("SlidingWindowCheckMapper", 10), + SLIDING_WINDOW_CHECK_PRINT_SINK("SlidingWindowCheckPrintSink", 11), + RESULT_TYPE_QUERYABLE_MAPPER_WITH_CUSTOM_SER("ResultTypeQueryableMapWithCustomStatefulTypeSerializer", 12), + MAPPER_RETURNS_OUT_WITH_CUSTOM_SER("MapReturnsOutputWithCustomStatefulTypeSerializer", 13), + EVENT_IDENTITY_MAPPER("EventIdentityMapper", 14); + + private final String name; + private final String uid; + + TestOperatorEnum(String name, int uid) { + this.name = name; + this.uid = String.format("%04d", uid); + } + + public String getName() { + return name; + } + + public String getUid() { + return uid; + } +} diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/builder/ArtificialStateBuilder.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/builder/ArtificialStateBuilder.java index aed94babff5aa..ac24f3946791f 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/builder/ArtificialStateBuilder.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/artificialstate/builder/ArtificialStateBuilder.java @@ -30,9 +30,9 @@ public abstract class ArtificialStateBuilder implements Serializable { private static final long serialVersionUID = -5887676929924485788L; - protected final String stateName; + final String stateName; - public ArtificialStateBuilder(String stateName) { + ArtificialStateBuilder(String stateName) { this.stateName = stateName; } From d099898b63db56a23e69ca92e0360c34cfb964d5 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Mar 2019 13:44:17 +0100 Subject: [PATCH 327/359] [hotfix][core] Fix Tuple0Serializer handling of null Tuple0Serializer serialized null as just a proper Tuple0 instance, which if later deserialized resulted in a regular value rather than null. This commit changes it so that it is no longer possible to serialize null with Tuple0Serializer. --- .../flink/api/java/typeutils/runtime/Tuple0Serializer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java index b96126d2f64b5..e173d7b3bac19 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java @@ -19,6 +19,7 @@ import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; @Internal public class Tuple0Serializer extends TupleSerializer { @@ -72,6 +73,7 @@ public int getLength() { @Override public void serialize(Tuple0 record, DataOutputView target) throws IOException { + Preconditions.checkNotNull(record); target.writeByte(42); } From 13b3e6b1557fc7d909c8b9e465c3361867fc3f3d Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 4 Mar 2019 12:12:54 +0100 Subject: [PATCH 328/359] [hotfix][core] Comparing whole collections rather than contents in KryoGenericTypeSerializerTest --- .../api/scala/runtime/KryoGenericTypeSerializerTest.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala index 1062e654e0839..a0c285fa8bf76 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala @@ -102,14 +102,14 @@ class KryoGenericTypeSerializerTest { def testScalaListSerialization(): Unit = { val a = List(42,1,49,1337) - runTests(a) + runTests(Seq(a)) } @Test def testScalaMutablelistSerialization(): Unit = { val a = scala.collection.mutable.ListBuffer(42,1,49,1337) - runTests(a) + runTests(Seq(a)) } @Test @@ -133,7 +133,7 @@ class KryoGenericTypeSerializerTest { val c = ComplexType("1337", 1, List(1)) val list = List(a, b, c) - runTests(list) + runTests(Seq(list)) } @Test @@ -143,7 +143,7 @@ class KryoGenericTypeSerializerTest { val c = new DerivedType2("bar", "foo") val list = List(a,b,c) - runTests(list) + runTests(Seq(list)) } From c3b49eec890e570bac98bf407c1ab682749fa8c5 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Sun, 3 Mar 2019 10:43:09 +0100 Subject: [PATCH 329/359] [hotfix][tests] Call all methods from SerializerTestBase in SerializerTestInstance by reflection --- .../typeutils/SerializerTestInstance.java | 42 ++++++++++++------- 1 file changed, 26 insertions(+), 16 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java index a7207582ff4b8..12c1f4dc5209f 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java @@ -19,6 +19,11 @@ package org.apache.flink.api.common.typeutils; +import org.junit.Test; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + public class SerializerTestInstance extends SerializerTestBase { private final TypeSerializer serializer; @@ -38,9 +43,9 @@ public SerializerTestInstance(TypeSerializer serializer, Class typeClass, this.length = length; this.testData = testData; } - + // -------------------------------------------------------------------------------------------- - + @Override protected TypeSerializer createSerializer() { return this.serializer; @@ -60,21 +65,26 @@ protected Class getTypeClass() { protected T[] getTestData() { return this.testData; } - + // -------------------------------------------------------------------------------------------- - + public void testAll() { - testInstantiate(); - testGetLength(); - testCopy(); - testCopyIntoNewElements(); - testCopyIntoReusedElements(); - testSerializeIndividually(); - testSerializeIndividuallyReusingValues(); - testSerializeAsSequenceNoReuse(); - testSerializeAsSequenceReusingValues(); - testSerializedCopyIndividually(); - testSerializedCopyAsSequence(); - testSerializabilityAndEquals(); + for (Method method : SerializerTestBase.class.getMethods()) { + if (method.getAnnotation(Test.class) == null) { + continue; + } + try { + method.invoke(this); + } catch (IllegalAccessException e) { + throw new RuntimeException("Unable to invoke test " + method.getName(), e); + } catch (InvocationTargetException e) { + sneakyThrow(e.getCause()); + } + } + } + + @SuppressWarnings("unchecked") + private static void sneakyThrow(Throwable e) throws E { + throw (E) e; } } From 54cf610494c0601ddbfde91ea62a07dc080feeb1 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Mar 2019 08:50:05 +0100 Subject: [PATCH 330/359] [FLINK-11420][core] Fixed duplicate method of TraversableSerializer The duplicate method of TypeSerializer used an equality check rather than reference check of the element serializer to decide if we need a deep copy. This commit uses proper reference comparison. --- .../flink/api/scala/typeutils/TraversableSerializer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala index b5d069fbe9396..71a482aba1db4 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala @@ -40,7 +40,7 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E]( override def duplicate = { val duplicateElementSerializer = elementSerializer.duplicate() - if (duplicateElementSerializer == elementSerializer) { + if (duplicateElementSerializer eq elementSerializer) { // is not stateful, so return ourselves this } else { From f4c0991bbebf651cf7bc72ecdb57cf74b07e8c5b Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Mar 2019 16:43:07 +0100 Subject: [PATCH 331/359] [hotfix][tests] Added matcher that performs deep comparison with taking Tuples into account This commit introduced a matcher that performs a deepEquals comparison similarly to Objects#deepEquals. The only difference is that it also performs deep equality check for some flink specific classes such as e.g. Tuples, Rows. --- .../typeutils/CompositeSerializerTest.java | 6 - .../common/typeutils/SerializerTestBase.java | 82 ++++---- .../typeutils/SerializerTestInstance.java | 13 ++ .../typeutils/runtime/RowSerializerTest.java | 14 +- .../runtime/TupleSerializerTest.java | 3 +- .../runtime/TupleSerializerTestInstance.java | 79 -------- .../testutils/CustomEqualityMatcher.java | 70 +++++++ .../flink/testutils/DeeplyEqualsChecker.java | 175 ++++++++++++++++++ .../ByteValueArraySerializerTest.java | 3 +- .../CharValueArraySerializerTest.java | 3 +- .../DoubleValueArraySerializerTest.java | 3 +- .../FloatValueArraySerializerTest.java | 3 +- .../IntValueArraySerializerTest.java | 3 +- .../LongValueArraySerializerTest.java | 3 +- .../NullValueArraySerializerTest.java | 3 +- .../ShortValueArraySerializerTest.java | 3 +- .../StringValueArraySerializerTest.java | 3 +- .../ValueArraySerializerTestBase.java | 39 ++++ .../ScalaSpecialTypesSerializerTest.scala | 83 ++++++--- .../runtime/TraversableSerializerTest.scala | 17 -- .../runtime/TupleSerializerTestInstance.scala | 81 ++++---- 21 files changed, 443 insertions(+), 246 deletions(-) delete mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java create mode 100644 flink-core/src/test/java/org/apache/flink/testutils/CustomEqualityMatcher.java create mode 100644 flink-core/src/test/java/org/apache/flink/testutils/DeeplyEqualsChecker.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArraySerializerTestBase.java diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeSerializerTest.java index fc5c241c20566..ab3e6f3a84c49 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/CompositeSerializerTest.java @@ -37,8 +37,6 @@ import java.util.function.IntFunction; import java.util.stream.IntStream; -import static org.junit.Assert.assertEquals; - /** Test suite for {@link CompositeSerializer}. */ public class CompositeSerializerTest { private static final ExecutionConfig execConf = new ExecutionConfig(); @@ -205,9 +203,5 @@ private static class CompositeSerializerTestInstance extends SerializerTestInsta private static Class> getCls(List instance) { return TypeExtractor.getForObject(instance).getTypeClass(); } - - protected void deepEquals(String message, List should, List is) { - assertEquals(message, should, is); - } } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java index 151dafb2f8413..f57b99b17816a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java @@ -25,8 +25,9 @@ import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.testutils.CustomEqualityMatcher; +import org.apache.flink.testutils.DeeplyEqualsChecker; import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; import org.apache.commons.lang3.SerializationException; @@ -40,14 +41,12 @@ import java.io.DataOutputStream; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.Objects; import java.util.concurrent.CyclicBarrier; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -61,6 +60,16 @@ */ public abstract class SerializerTestBase extends TestLogger { + private final DeeplyEqualsChecker checker; + + protected SerializerTestBase() { + this.checker = new DeeplyEqualsChecker(); + } + + protected SerializerTestBase(DeeplyEqualsChecker checker) { + this.checker = checker; + } + protected abstract TypeSerializer createSerializer(); /** @@ -458,7 +467,8 @@ public void testDuplicate() throws Exception { startLatch, serializer.duplicate(), testData, - 120L); + 120L, + checker); runner.start(); concurrentRunners.add(runner); @@ -472,46 +482,8 @@ public void testDuplicate() throws Exception { // -------------------------------------------------------------------------------------------- - protected void deepEquals(String message, T should, T is) { - Assert.assertTrue((should == null && is == null) || (should != null && is != null)); - if (should == null) { - return; - } - if (should.getClass().isArray()) { - if (should instanceof boolean[]) { - Assert.assertTrue(message, Arrays.equals((boolean[]) should, (boolean[]) is)); - } - else if (should instanceof byte[]) { - assertArrayEquals(message, (byte[]) should, (byte[]) is); - } - else if (should instanceof short[]) { - assertArrayEquals(message, (short[]) should, (short[]) is); - } - else if (should instanceof int[]) { - assertArrayEquals(message, (int[]) should, (int[]) is); - } - else if (should instanceof long[]) { - assertArrayEquals(message, (long[]) should, (long[]) is); - } - else if (should instanceof float[]) { - assertArrayEquals(message, (float[]) should, (float[]) is, 0.0f); - } - else if (should instanceof double[]) { - assertArrayEquals(message, (double[]) should, (double[]) is, 0.0); - } - else if (should instanceof char[]) { - assertArrayEquals(message, (char[]) should, (char[]) is); - } - else { - assertArrayEquals(message, (Object[]) should, (Object[]) is); - } - } - else if (should instanceof Throwable) { - assertEquals(((Throwable)should).getMessage(), ((Throwable)is).getMessage()); - } - else { - assertEquals(message, should, is); - } + private void deepEquals(String message, T should, T is) { + assertThat(message, is, CustomEqualityMatcher.deeplyEquals(should).withChecker(checker)); } // -------------------------------------------------------------------------------------------- @@ -569,18 +541,20 @@ static class SerializerRunner extends Thread { final TypeSerializer serializer; final T[] testData; final long durationLimitMillis; - Exception failure; + Throwable failure; + final DeeplyEqualsChecker checker; SerializerRunner( CyclicBarrier allReadyBarrier, TypeSerializer serializer, T[] testData, - long testTargetDurationMillis) { + long testTargetDurationMillis, DeeplyEqualsChecker checker) { this.allReadyBarrier = allReadyBarrier; this.serializer = serializer; this.testData = testData; this.durationLimitMillis = testTargetDurationMillis; + this.checker = checker; this.failure = null; } @@ -602,8 +576,10 @@ public void run() { T copySerdeTestItem = serializer.copy(serdeTestItem); dataOutputSerializer.clear(); - Preconditions.checkState(Objects.deepEquals(testItem, copySerdeTestItem), - "Serialization/Deserialization cycle resulted in an object that are not equal to the original."); + assertThat( + "Serialization/Deserialization cycle resulted in an object that are not equal to the original.", + copySerdeTestItem, + CustomEqualityMatcher.deeplyEquals(testItem).withChecker(checker)); // try to enforce some upper bound to the test time if (System.nanoTime() >= endTimeNanos) { @@ -611,14 +587,18 @@ public void run() { } } } - } catch (Exception ex) { + } catch (Throwable ex) { failure = ex; } } void checkResult() throws Exception { if (failure != null) { - throw failure; + if (failure instanceof AssertionError) { + throw (AssertionError) failure; + } else { + throw (Exception) failure; + } } } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java index 12c1f4dc5209f..ec25e9e27287d 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestInstance.java @@ -19,6 +19,8 @@ package org.apache.flink.api.common.typeutils; +import org.apache.flink.testutils.DeeplyEqualsChecker; + import org.junit.Test; import java.lang.reflect.InvocationTargetException; @@ -38,6 +40,17 @@ public class SerializerTestInstance extends SerializerTestBase { @SafeVarargs public SerializerTestInstance(TypeSerializer serializer, Class typeClass, int length, T... testData) { + this(new DeeplyEqualsChecker(), serializer, typeClass, length, testData); + } + + @SafeVarargs + public SerializerTestInstance( + DeeplyEqualsChecker checker, + TypeSerializer serializer, + Class typeClass, + int length, + T... testData) { + super(checker); this.serializer = serializer; this.typeClass = typeClass; this.length = length; diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/RowSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/RowSerializerTest.java index d08d68a6678bc..f94943183839d 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/RowSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/RowSerializerTest.java @@ -27,12 +27,11 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.types.Row; + import org.junit.Test; import java.io.Serializable; -import static org.junit.Assert.assertEquals; - public class RowSerializerTest { @Test @@ -150,17 +149,6 @@ private class RowSerializerTestInstance extends SerializerTestInstance { Row... testData) { super(serializer, Row.class, -1, testData); } - - @Override - protected void deepEquals(String message, Row should, Row is) { - int arity = should.getArity(); - assertEquals(message, arity, is.getArity()); - for (int i = 0; i < arity; i++) { - Object copiedValue = should.getField(i); - Object element = is.getField(i); - assertEquals(message, element, copiedValue); - } - } } public static class MyPojo implements Serializable, Comparable { diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java index 13f91b048c617..f6cf2def30233 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java @@ -23,6 +23,7 @@ import java.util.Random; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.SerializerTestInstance; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple0; @@ -225,7 +226,7 @@ private void runTests(int length, T... instances) { if(tupleClass == Tuple0.class) { length = 1; } - TupleSerializerTestInstance test = new TupleSerializerTestInstance(serializer, tupleClass, length, instances); + SerializerTestInstance test = new SerializerTestInstance<>(serializer, tupleClass, length, instances); test.testAll(); } catch (Exception e) { diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java deleted file mode 100644 index a196984d4e513..0000000000000 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.flink.api.java.typeutils.runtime; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; - -import java.util.Arrays; - -import org.apache.flink.api.common.typeutils.SerializerTestInstance; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple; -import org.junit.Assert; - -public class TupleSerializerTestInstance extends SerializerTestInstance { - - public TupleSerializerTestInstance(TypeSerializer serializer, Class typeClass, int length, T[] testData) { - super(serializer, typeClass, length, testData); - } - - protected void deepEquals(String message, T shouldTuple, T isTuple) { - Assert.assertEquals(shouldTuple.getArity(), isTuple.getArity()); - - for (int i = 0; i < shouldTuple.getArity(); i++) { - Object should = shouldTuple.getField(i); - Object is = isTuple.getField(i); - - if (should.getClass().isArray()) { - if (should instanceof boolean[]) { - Assert.assertTrue(message, Arrays.equals((boolean[]) should, (boolean[]) is)); - } - else if (should instanceof byte[]) { - assertArrayEquals(message, (byte[]) should, (byte[]) is); - } - else if (should instanceof short[]) { - assertArrayEquals(message, (short[]) should, (short[]) is); - } - else if (should instanceof int[]) { - assertArrayEquals(message, (int[]) should, (int[]) is); - } - else if (should instanceof long[]) { - assertArrayEquals(message, (long[]) should, (long[]) is); - } - else if (should instanceof float[]) { - assertArrayEquals(message, (float[]) should, (float[]) is, 0.0f); - } - else if (should instanceof double[]) { - assertArrayEquals(message, (double[]) should, (double[]) is, 0.0); - } - else if (should instanceof char[]) { - assertArrayEquals(message, (char[]) should, (char[]) is); - } - else { - assertArrayEquals(message, (Object[]) should, (Object[]) is); - } - } - else { - assertEquals(message, should, is); - } - } - } -} diff --git a/flink-core/src/test/java/org/apache/flink/testutils/CustomEqualityMatcher.java b/flink-core/src/test/java/org/apache/flink/testutils/CustomEqualityMatcher.java new file mode 100644 index 0000000000000..e80666c6222cb --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/testutils/CustomEqualityMatcher.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.testutils; + +import org.apache.flink.api.java.tuple.Tuple; + +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; + +import java.util.Arrays; + +public class CustomEqualityMatcher extends BaseMatcher { + + private final Object wanted; + + private final DeeplyEqualsChecker checker; + + private CustomEqualityMatcher(Object wanted, DeeplyEqualsChecker checker) { + this.wanted = wanted; + this.checker = checker; + } + + /** + * This matcher performs similar comparison to {@link org.hamcrest.core.IsEqual}, which resembles + * {@link java.util.Objects#deepEquals(Object, Object)} logic. The only difference here is that {@link Tuple}s + * are treated similarly to arrays. + * + *

    This means that if we compare two Tuples that contain arrays, those arrays will + * be compared with {@link Arrays#deepEquals(Object[], Object[])} rather than with reference comparison. + * + * @param item expected value + */ + public static CustomEqualityMatcher deeplyEquals(Object item) { + return new CustomEqualityMatcher(item, new DeeplyEqualsChecker()); + } + + /** + * Performs assertions with this customly configured {@link DeeplyEqualsChecker}. It might have some additional + * rules applied. + */ + public CustomEqualityMatcher withChecker(DeeplyEqualsChecker checker) { + return new CustomEqualityMatcher(wanted, checker); + } + + @Override + public boolean matches(Object item) { + return checker.deepEquals(item, wanted); + } + + @Override + public void describeTo(Description description) { + description.appendValue(wanted); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/testutils/DeeplyEqualsChecker.java b/flink-core/src/test/java/org/apache/flink/testutils/DeeplyEqualsChecker.java new file mode 100644 index 0000000000000..00119f161153a --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/testutils/DeeplyEqualsChecker.java @@ -0,0 +1,175 @@ +/* + * 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.testutils; + +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.types.Row; + +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.function.BiFunction; + +/** + * Deep equality checker for tests. It performs deep checks for objects which have no proper deepEquals methods like: + *

      + *
    • {@link Tuple}s
    • + *
    • {@link Iterable}s
    • + *
    • Java arrays
    • + *
    • {@link Row}
    • + *
    • {@link Throwable}
    • + *
    + * + *

    One can also provide custom check for additional categories of objects with + * {@link DeeplyEqualsChecker#withCustomCheck(BiFunction, CustomEqualityChecker)}. This is used e.g. in scala's tests. + */ +public class DeeplyEqualsChecker { + + /** + * Checker that compares o1 and o2 objects if they are deeply equal. + * + *

    NOTE: All nested comparisons should be done through checker. + */ + public interface CustomEqualityChecker { + boolean check(Object o1, Object o2, DeeplyEqualsChecker checker); + } + + private final List, CustomEqualityChecker>> customCheckers + = new ArrayList<>(); + + /** + * Adds custom check. Those check are always performed first, only after that it fallbacks to default checks. + * + * @param shouldCheck function to evaluate if the objects should be compared with comparator + * @param comparator to perform equality comparison if the shouldCheck passed + * @return checker with added custom checks + */ + public DeeplyEqualsChecker withCustomCheck( + BiFunction shouldCheck, + CustomEqualityChecker comparator) { + customCheckers.add(Tuple2.of(shouldCheck, comparator)); + return this; + } + + public boolean deepEquals(Object o1, Object o2) { + if (o1 == o2) { + return true; + } else if (o1 == null || o2 == null) { + return false; + } else { + return customCheck(o1, o2).orElseGet(() -> deepEquals0(o1, o2)); + } + } + + private Optional customCheck(Object o1, Object o2) { + return customCheckers.stream() + .filter(checker -> checker.f0.apply(o1, o2)) + .findAny() + .map(checker -> checker.f1.check(o1, o2, this)); + } + + private boolean deepEquals0(Object e1, Object e2) { + if (e1.getClass().isArray() && e2.getClass().isArray()) { + return deepEqualsArray(e1, e2); + } else if (e1 instanceof Iterable && e2 instanceof Iterable) { + return deepEqualsIterable((Iterable) e1, (Iterable) e2); + } else if (e1 instanceof Tuple && e2 instanceof Tuple) { + return deepEqualsTuple((Tuple) e1, (Tuple) e2); + } else if (e1 instanceof Row && e2 instanceof Row) { + return deepEqualsRow((Row) e1, (Row) e2); + } else if (e1 instanceof Throwable && e2 instanceof Throwable) { + return ((Throwable) e1).getMessage().equals(((Throwable) e2).getMessage()); + } else { + return e1.equals(e2); + } + } + + private boolean deepEqualsIterable(Iterable i1, Iterable i2) { + Iterator s1 = i1.iterator(); + Iterator s2 = i2.iterator(); + + while (s1.hasNext() && s2.hasNext()) { + Object l = s1.next(); + Object r = s2.next(); + + if (!deepEquals(l, r)) { + return false; + } + } + return !s1.hasNext() && !s2.hasNext(); + } + + private boolean deepEqualsTuple(Tuple tuple1, Tuple tuple2) { + if (tuple1.getArity() != tuple2.getArity()) { + return false; + } + + for (int i = 0; i < tuple1.getArity(); i++) { + Object o1 = tuple1.getField(i); + Object o2 = tuple2.getField(i); + + if (!deepEquals(o1, o2)) { + return false; + } + } + + return true; + } + + private boolean deepEqualsArray(Object array1, Object array2) { + int length1 = Array.getLength(array1); + int length2 = Array.getLength(array2); + + if (length1 != length2) { + return false; + } + + for (int i = 0; i < length1; i++) { + Object o1 = Array.get(array1, i); + Object o2 = Array.get(array2, i); + + if (!deepEquals(o1, o2)) { + return false; + } + } + + return true; + } + + private boolean deepEqualsRow(Row row1, Row row2) { + int arity = row1.getArity(); + + if (row1.getArity() != row2.getArity()) { + return false; + } + + for (int i = 0; i < arity; i++) { + Object copiedValue = row1.getField(i); + Object element = row2.getField(i); + if (!deepEquals(copiedValue, element)) { + return false; + } + } + + return true; + } +} diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ByteValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ByteValueArraySerializerTest.java index 4a29318169a56..2f8f8fcf70973 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ByteValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ByteValueArraySerializerTest.java @@ -19,7 +19,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.ByteValue; @@ -28,7 +27,7 @@ /** * A test for the {@link ByteValueArraySerializer}. */ -public class ByteValueArraySerializerTest extends SerializerTestBase { +public class ByteValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/CharValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/CharValueArraySerializerTest.java index 86ee10dbf0d05..432f146bf9940 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/CharValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/CharValueArraySerializerTest.java @@ -19,7 +19,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.CharValue; @@ -28,7 +27,7 @@ /** * A test for the {@link CharValueArraySerializer}. */ -public class CharValueArraySerializerTest extends SerializerTestBase { +public class CharValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/DoubleValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/DoubleValueArraySerializerTest.java index 49c1f65cb2194..f0396d091ae21 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/DoubleValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/DoubleValueArraySerializerTest.java @@ -19,7 +19,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.DoubleValue; @@ -28,7 +27,7 @@ /** * A test for the {@link DoubleValueArraySerializer}. */ -public class DoubleValueArraySerializerTest extends SerializerTestBase { +public class DoubleValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/FloatValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/FloatValueArraySerializerTest.java index 14312c3550604..64a58fe8a6e25 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/FloatValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/FloatValueArraySerializerTest.java @@ -19,7 +19,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.FloatValue; @@ -28,7 +27,7 @@ /** * A test for the {@link FloatValueArraySerializer}. */ -public class FloatValueArraySerializerTest extends SerializerTestBase { +public class FloatValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java index 88a367c4e01fb..21a9bcb29c06c 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializerTest.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.IntValue; @@ -27,7 +26,7 @@ /** * A test for the {@link IntValueArraySerializer}. */ -public class IntValueArraySerializerTest extends SerializerTestBase { +public class IntValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java index 50201daf6c7a1..e617b57143170 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializerTest.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.LongValue; @@ -27,7 +26,7 @@ /** * A test for the {@link LongValueArraySerializer}. */ -public class LongValueArraySerializerTest extends SerializerTestBase { +public class LongValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java index 8cf1efa37308a..8b30cddd185d6 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/NullValueArraySerializerTest.java @@ -18,14 +18,13 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.NullValue; /** * A test for the {@link NullValueArraySerializer}. */ -public class NullValueArraySerializerTest extends SerializerTestBase { +public class NullValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ShortValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ShortValueArraySerializerTest.java index 005f4d1e254f5..ad5ce8b551cef 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ShortValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ShortValueArraySerializerTest.java @@ -19,7 +19,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.ShortValue; @@ -28,7 +27,7 @@ /** * A test for the {@link ShortValueArraySerializer}. */ -public class ShortValueArraySerializerTest extends SerializerTestBase { +public class ShortValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java index 52892caf15c97..7970025164620 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializerTest.java @@ -18,7 +18,6 @@ package org.apache.flink.graph.types.valuearray; -import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.StringValue; @@ -27,7 +26,7 @@ /** * A test for the {@link StringValueArraySerializer}. */ -public class StringValueArraySerializerTest extends SerializerTestBase { +public class StringValueArraySerializerTest extends ValueArraySerializerTestBase { @Override protected TypeSerializer createSerializer() { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArraySerializerTestBase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArraySerializerTestBase.java new file mode 100644 index 0000000000000..81aa3a11c8dd3 --- /dev/null +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/types/valuearray/ValueArraySerializerTestBase.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.types.valuearray; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.testutils.DeeplyEqualsChecker; + +import java.util.Objects; + +/** + * Base class for tests of {@link org.apache.flink.api.common.typeutils.TypeSerializer}s for any {@link ValueArray}. + * It overrides default deepEquals of {@link Iterable}s with {@link Objects#equals(Object, Object)}. + */ +public abstract class ValueArraySerializerTestBase> extends SerializerTestBase { + ValueArraySerializerTestBase() { + super( + new DeeplyEqualsChecker() + .withCustomCheck( + (o1, o2) -> o1 instanceof ValueArray && o2 instanceof ValueArray, + (o1, o2, checker) -> Objects.equals(o1, o2))); + } + +} diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala index 555359f89abf6..8809e777acb49 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala @@ -17,11 +17,16 @@ */ package org.apache.flink.api.scala.runtime +import java.lang.{Boolean => JBoolean} +import java.util.function.BiFunction + import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.{SerializerTestInstance, TypeSerializer} import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer import org.apache.flink.api.scala._ +import org.apache.flink.testutils.DeeplyEqualsChecker +import org.apache.flink.testutils.DeeplyEqualsChecker.CustomEqualityChecker import org.junit.Assert._ import org.junit.{Assert, Test} @@ -143,12 +148,67 @@ class ScalaSpecialTypesSerializerTest { } } +object ScalaSpecialTypesSerializerTestInstance { + + val isTraversable: BiFunction[AnyRef, AnyRef, JBoolean] = + new BiFunction[AnyRef, AnyRef, JBoolean] { + override def apply(o1: scala.AnyRef, o2: scala.AnyRef): JBoolean = + o1.isInstanceOf[TraversableOnce[_]] && o2.isInstanceOf[TraversableOnce[_]] + } + + val isFailure: BiFunction[AnyRef, AnyRef, JBoolean] = + new BiFunction[AnyRef, AnyRef, JBoolean] { + override def apply(o1: scala.AnyRef, o2: scala.AnyRef): JBoolean = + o1.isInstanceOf[Failure[_]] && o2.isInstanceOf[Failure[_]] + } + + val compareTraversable: CustomEqualityChecker = + new CustomEqualityChecker { + override def check( + o1: AnyRef, + o2: AnyRef, + checker: DeeplyEqualsChecker): Boolean = { + val s1 = o1.asInstanceOf[TraversableOnce[_]].toIterator + val s2 = o2.asInstanceOf[TraversableOnce[_]].toIterator + + while (s1.hasNext && s2.hasNext) { + val l = s1.next + val r = s2.next + if (!checker.deepEquals(l, r)) { + return false + } + } + !s1.hasNext && !s2.hasNext + } + } + + val compareFailure: CustomEqualityChecker = + new CustomEqualityChecker { + override def check( + o1: AnyRef, + o2: AnyRef, + checker: DeeplyEqualsChecker): Boolean = { + o1.asInstanceOf[Failure[_]].exception.getMessage + .equals(o2.asInstanceOf[Failure[_]].exception.getMessage) + } + } +} + class ScalaSpecialTypesSerializerTestInstance[T]( serializer: TypeSerializer[T], typeClass: Class[T], length: Int, testData: Array[T]) - extends SerializerTestInstance[T](serializer, typeClass, length, testData: _*) { + extends SerializerTestInstance[T]( + new DeeplyEqualsChecker() + .withCustomCheck(ScalaSpecialTypesSerializerTestInstance.isTraversable, + ScalaSpecialTypesSerializerTestInstance.compareTraversable) + .withCustomCheck(ScalaSpecialTypesSerializerTestInstance.isFailure, + ScalaSpecialTypesSerializerTestInstance.compareFailure), + serializer, + typeClass, + length, + testData: _*) { @Test override def testInstantiate(): Unit = { @@ -173,27 +233,6 @@ class ScalaSpecialTypesSerializerTestInstance[T]( } } } - - override protected def deepEquals(message: String, should: T, is: T) { - should match { - case trav: TraversableOnce[_] => - val isTrav = is.asInstanceOf[TraversableOnce[_]] - assertEquals(message, trav.size, isTrav.size) - val it = trav.toIterator - val isIt = isTrav.toIterator - while (it.hasNext) { - val should = it.next() - val is = isIt.next() - assertEquals(message, should, is) - } - - case Failure(t) => - is.asInstanceOf[Failure[_]].exception.equals(t) - - case _ => - super.deepEquals(message, should, is) - } - } } object WeekDay extends Enumeration { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala index e177e7c4ac42f..3df28ed5816d6 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala @@ -177,22 +177,5 @@ class TraversableSerializerTestInstance[T]( } } - override protected def deepEquals(message: String, should: T, is: T) { - should match { - case trav: TraversableOnce[_] => - val isTrav = is.asInstanceOf[TraversableOnce[_]] - assertEquals(message, trav.size, isTrav.size) - val it = trav.toIterator - val isIt = isTrav.toIterator - while (it.hasNext) { - val should = it.next() - val is = isIt.next() - assertEquals(message, should, is) - } - - case _ => - super.deepEquals(message, should, is) - } - } } diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala index 7a425900c96c9..3bdc54aec9118 100644 --- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTestInstance.scala @@ -17,19 +17,57 @@ */ package org.apache.flink.api.scala.runtime +import java.lang.{Boolean => JBoolean} +import java.util.function.BiFunction + +import org.apache.flink.api.common.typeutils.{SerializerTestInstance, TypeSerializer} +import org.apache.flink.testutils.DeeplyEqualsChecker +import org.apache.flink.testutils.DeeplyEqualsChecker.CustomEqualityChecker import org.junit.Assert._ -import org.apache.flink.api.common.typeutils.SerializerTestInstance -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.junit.Assert import org.junit.Test +object TupleSerializerTestInstance { + val isProduct: BiFunction[AnyRef, AnyRef, JBoolean] = + new BiFunction[AnyRef, AnyRef, JBoolean] { + override def apply(o1: scala.AnyRef, o2: scala.AnyRef): JBoolean = + o1.isInstanceOf[Product] && o2.isInstanceOf[Product] + } + + val compareProduct: CustomEqualityChecker = + new CustomEqualityChecker { + override def check( + o1: AnyRef, + o2: AnyRef, + checker: DeeplyEqualsChecker): Boolean = { + val p1 = o1.asInstanceOf[Product].productIterator + val p2 = o2.asInstanceOf[Product].productIterator + + while (p1.hasNext && p2.hasNext) { + val l = p1.next + val r = p2.next + if (!checker.deepEquals(l, r)) { + return false + } + } + !p1.hasNext && !p2.hasNext + } + } +} + class TupleSerializerTestInstance[T <: Product] ( serializer: TypeSerializer[T], typeClass: Class[T], length: Int, testData: Array[T]) - extends SerializerTestInstance[T](serializer, typeClass, length, testData: _*) { + extends SerializerTestInstance[T]( + new DeeplyEqualsChecker() + .withCustomCheck(TupleSerializerTestInstance.isProduct, + TupleSerializerTestInstance.compareProduct), + serializer, + typeClass, + length, + testData: _*) { @Test override def testInstantiate(): Unit = { @@ -51,40 +89,5 @@ class TupleSerializerTestInstance[T <: Product] ( } } } - - protected override def deepEquals(message: String, shouldTuple: T, isTuple: T) { - Assert.assertEquals(shouldTuple.productArity, isTuple.productArity) - for (i <- 0 until shouldTuple.productArity) { - val should = shouldTuple.productElement(i) - val is = isTuple.productElement(i) - if (should.getClass.isArray) { - should match { - case booleans: Array[Boolean] => - Assert.assertTrue(message, booleans.sameElements(is.asInstanceOf[Array[Boolean]])) - case bytes: Array[Byte] => - assertArrayEquals(message, bytes, is.asInstanceOf[Array[Byte]]) - case shorts: Array[Short] => - assertArrayEquals(message, shorts, is.asInstanceOf[Array[Short]]) - case ints: Array[Int] => - assertArrayEquals(message, ints, is.asInstanceOf[Array[Int]]) - case longs: Array[Long] => - assertArrayEquals(message, longs, is.asInstanceOf[Array[Long]]) - case floats: Array[Float] => - assertArrayEquals(message, floats, is.asInstanceOf[Array[Float]], 0.0f) - case doubles: Array[Double] => - assertArrayEquals(message, doubles, is.asInstanceOf[Array[Double]], 0.0) - case chars: Array[Char] => - assertArrayEquals(message, chars, is.asInstanceOf[Array[Char]]) - case _ => - assertArrayEquals( - message, - should.asInstanceOf[Array[AnyRef]], - is.asInstanceOf[Array[AnyRef]]) - } - } else { - assertEquals(message, should, is) - } - } - } } From 35778da58c2cdc0a88997a4184b9235bbbc120c7 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 4 Mar 2019 12:11:12 +0100 Subject: [PATCH 332/359] [hotfix][core] Added snapshot for NothingSerializerSnapshot --- .../flink/api/scala/typeutils/NothingSerializer.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala index eff57b68755b7..d779afa9d1a23 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.common.typeutils._ import org.apache.flink.core.memory.{DataInputView, DataOutputView} /** @@ -56,8 +56,8 @@ class NothingSerializer extends TypeSerializer[Any] { override def deserialize(reuse: Any, source: DataInputView): Any = throw new RuntimeException("This must not be used. You encountered a bug.") - override def snapshotConfiguration(): TypeSerializerConfigSnapshot[Any] = - throw new RuntimeException("This must not be used. You encountered a bug.") + override def snapshotConfiguration(): TypeSerializerSnapshot[Any] = + new NothingSerializerSnapshot override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot[_]): CompatibilityResult[Any] = @@ -78,3 +78,7 @@ class NothingSerializer extends TypeSerializer[Any] { classOf[NothingSerializer].hashCode() } } + +class NothingSerializerSnapshot + extends SimpleTypeSerializerSnapshot[Any](classOf[NothingSerializer]) { +} From 70bc26c330bb719d5fec3745605eaf5f530d18f0 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 5 Mar 2019 11:00:07 +0100 Subject: [PATCH 333/359] [FLINK-11823][core] Fixed duplicate method of TrySerializer --- .../api/scala/typeutils/TrySerializer.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala index 786c12839381c..03324568bb42f 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala @@ -32,12 +32,21 @@ import scala.util.{Failure, Success, Try} @SerialVersionUID(-3052182891252564491L) class TrySerializer[A]( private val elemSerializer: TypeSerializer[A], - private val executionConfig: ExecutionConfig) + private val throwableSerializer: TypeSerializer[Throwable]) extends TypeSerializer[Try[A]] { - override def duplicate: TrySerializer[A] = this + private[typeutils] def this(elemSerializer: TypeSerializer[A], + executionConfig: ExecutionConfig) = { + this( + elemSerializer, + new KryoSerializer[Throwable](classOf[Throwable], executionConfig) + ) + } - val throwableSerializer = new KryoSerializer[Throwable](classOf[Throwable], executionConfig) + override def duplicate: TrySerializer[A] = new TrySerializer[A]( + elemSerializer.duplicate(), + throwableSerializer.duplicate() + ) override def createInstance: Try[A] = { Failure(new RuntimeException("Empty Failure")) @@ -97,7 +106,7 @@ class TrySerializer[A]( } override def hashCode(): Int = { - 31 * elemSerializer.hashCode() + executionConfig.hashCode() + 31 * elemSerializer.hashCode() + throwableSerializer.hashCode() } // -------------------------------------------------------------------------------------------- From fa53d747c6716afe451cf015778794148b049a87 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 12 Feb 2019 15:18:16 +0100 Subject: [PATCH 334/359] [FLINK-11542][config][docs] Extend memory configuration descriptions --- .../generated/task_manager_configuration.html | 25 ---------- .../task_manager_memory_configuration.html | 36 ++++++++++++++ docs/ops/config.md | 6 +++ .../configuration/TaskManagerOptions.java | 47 ++++++++++++++----- 4 files changed, 77 insertions(+), 37 deletions(-) create mode 100644 docs/_includes/generated/task_manager_memory_configuration.html diff --git a/docs/_includes/generated/task_manager_configuration.html b/docs/_includes/generated/task_manager_configuration.html index 63bfb043ea44a..c11faa61139d3 100644 --- a/docs/_includes/generated/task_manager_configuration.html +++ b/docs/_includes/generated/task_manager_configuration.html @@ -67,31 +67,6 @@ false Whether to kill the TaskManager when the task thread throws an OutOfMemoryError. - -

    taskmanager.memory.fraction
    - 0.7 - The relative amount of memory (after subtracting the amount of memory used by network buffers) that the task manager reserves for sorting, hash tables, and caching of intermediate results. For example, a value of `0.8` means that a task manager reserves 80% of its memory for internal data buffers, leaving 20% of free memory for the task manager's heap for objects created by user-defined functions. This parameter is only evaluated, if taskmanager.memory.size is not set. - - -
    taskmanager.memory.off-heap
    - false - Memory allocation method (JVM heap or off-heap), used for managed memory of the TaskManager as well as the network buffers. - - -
    taskmanager.memory.preallocate
    - false - Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting. - - -
    taskmanager.memory.segment-size
    - "32kb" - Size of memory buffers used by the network stack and the memory manager. - - -
    taskmanager.memory.size
    - "0" - Amount of memory to be allocated by the task manager's memory manager. If not set, a relative fraction will be allocated. -
    taskmanager.network.detailed-metrics
    false diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html new file mode 100644 index 0000000000000..306a1fd378307 --- /dev/null +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    KeyDefaultDescription
    taskmanager.memory.fraction
    0.7The relative amount of memory (after subtracting the amount of memory used by network buffers) that the task manager reserves for sorting, hash tables, and caching of intermediate results. For example, a value of `0.8` means that a task manager reserves 80% of its memory (on-heap or off-heap depending on taskmanager.memory.off-heap) for internal data buffers, leaving 20% of free memory for the task manager's heap for objects created by user-defined functions. This parameter is only evaluated, if taskmanager.memory.size is not set.
    taskmanager.memory.off-heap
    falseMemory allocation method (JVM heap or off-heap), used for managed memory of the TaskManager. For setups with larger quantities of memory, this can improve the efficiency of the operations performed on the memory.
    When set to true, then it is advised that `taskmanager.memory.preallocate` is also set to true.
    taskmanager.memory.preallocate
    falseWhether TaskManager managed memory should be pre-allocated when the TaskManager is starting. When `taskmanager.memory.off-heap` is set to true, then it is advised that this configuration is also set to true. If this configuration is set to false cleaning up of the allocated off-heap memory happens only when the configured JVM parameter MaxDirectMemorySize is reached by triggering a full GC. For streaming setups is is highly recommended to set this value to false as the core state backends currently do not use the managed memory.
    taskmanager.memory.segment-size
    "32kb"Size of memory buffers used by the network stack and the memory manager.
    taskmanager.memory.size
    "0"The amount of memory (in megabytes) that the task manager reserves on-heap or off-heap (depending on taskmanager.memory.off-heap) for sorting, hash tables, and caching of intermediate results. If unspecified, the memory manager will take a fixed ratio with respect to the size of the task manager JVM as specified by taskmanager.memory.fraction.
    diff --git a/docs/ops/config.md b/docs/ops/config.md index 79e768c394603..e80603027e16f 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -70,6 +70,12 @@ These parameters configure the default HDFS used by Flink. Setups that do not sp {% include generated/task_manager_configuration.html %} +For *batch* jobs (or if `taskmanager.memoy.preallocate` is enabled) Flink allocates a fraction of 0.7 of the free memory (total memory configured via taskmanager.heap.mb minus memory used for network buffers) for its managed memory. Managed memory helps Flink to run the batch operators efficiently. It prevents OutOfMemoryExceptions because Flink knows how much memory it can use to execute operations. If Flink runs out of managed memory, it utilizes disk space. Using managed memory, some operations can be performed directly on the raw data without having to deserialize the data to convert it into Java objects. All in all, managed memory improves the robustness and speed of the system. + +The default fraction for managed memory can be adjusted using the taskmanager.memory.fraction parameter. An absolute value may be set using taskmanager.memory.size (overrides the fraction parameter). If desired, the managed memory may be allocated outside the JVM heap. This may improve performance in setups with large memory sizes. + +{% include generated/task_manager_memory_configuration.html %} + ### Distributed Coordination (via Akka) {% include generated/akka_configuration.html %} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index 6d3ac4515b20f..61331b28d57b7 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -19,16 +19,23 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.ConfigGroup; +import org.apache.flink.annotation.docs.ConfigGroups; import org.apache.flink.annotation.docs.Documentation; +import org.apache.flink.configuration.description.Description; import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.configuration.description.TextElement.code; /** * The set of configuration options relating to TaskManager and Task settings. */ @PublicEvolving +@ConfigGroups(groups = @ConfigGroup(name = "TaskManagerMemory", keyPrefix = "taskmanager.memory")) public class TaskManagerOptions { + private static final String MANAGED_MEMORY_PRE_ALLOCATE_KEY = "taskmanager.memory.preallocate"; + // ------------------------------------------------------------------------ // General TaskManager Options // ------------------------------------------------------------------------ @@ -204,8 +211,10 @@ public class TaskManagerOptions { public static final ConfigOption MANAGED_MEMORY_SIZE = key("taskmanager.memory.size") .defaultValue("0") - .withDescription("Amount of memory to be allocated by the task manager's memory manager." + - " If not set, a relative fraction will be allocated."); + .withDescription("The amount of memory (in megabytes) that the task manager reserves on-heap or off-heap" + + " (depending on taskmanager.memory.off-heap) for sorting, hash tables, and caching of intermediate" + + " results. If unspecified, the memory manager will take a fixed ratio with respect to the size of" + + " the task manager JVM as specified by taskmanager.memory.fraction."); /** * Fraction of free memory allocated by the memory manager if {@link #MANAGED_MEMORY_SIZE} is @@ -214,12 +223,15 @@ public class TaskManagerOptions { public static final ConfigOption MANAGED_MEMORY_FRACTION = key("taskmanager.memory.fraction") .defaultValue(0.7f) - .withDescription("The relative amount of memory (after subtracting the amount of memory used by network" + - " buffers) that the task manager reserves for sorting, hash tables, and caching of intermediate results." + - " For example, a value of `0.8` means that a task manager reserves 80% of its memory" + - " for internal data buffers, leaving 20% of free memory for the task manager's heap for objects" + - " created by user-defined functions. This parameter is only evaluated, if " + MANAGED_MEMORY_SIZE.key() + - " is not set."); + .withDescription(Description.builder() + .text("The relative amount of memory (after subtracting the amount of memory used by network" + + " buffers) that the task manager reserves for sorting, hash tables, and caching of intermediate results." + + " For example, a value of %s means that a task manager reserves 80% of its memory" + + " (on-heap or off-heap depending on taskmanager.memory.off-heap)" + + " for internal data buffers, leaving 20% of free memory for the task manager's heap for objects" + + " created by user-defined functions. This parameter is only evaluated, if " + MANAGED_MEMORY_SIZE.key() + + " is not set.", code("0.8")) + .build()); /** * Memory allocation method (JVM heap or off-heap), used for managed memory of the TaskManager @@ -228,16 +240,27 @@ public class TaskManagerOptions { public static final ConfigOption MEMORY_OFF_HEAP = key("taskmanager.memory.off-heap") .defaultValue(false) - .withDescription("Memory allocation method (JVM heap or off-heap), used for managed memory of the" + - " TaskManager as well as the network buffers."); + .withDescription(Description.builder() + .text("Memory allocation method (JVM heap or off-heap), used for managed memory of the" + + " TaskManager. For setups with larger quantities of memory, this can" + + " improve the efficiency of the operations performed on the memory.") + .linebreak() + .text("When set to true, then it is advised that %s is also set to true.", code(MANAGED_MEMORY_PRE_ALLOCATE_KEY)) + .build()); /** * Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting. */ public static final ConfigOption MANAGED_MEMORY_PRE_ALLOCATE = - key("taskmanager.memory.preallocate") + key(MANAGED_MEMORY_PRE_ALLOCATE_KEY) .defaultValue(false) - .withDescription("Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting."); + .withDescription(Description.builder() + .text("Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting." + + " When %s is set to true, then it is advised that this configuration is also" + + " set to true. If this configuration is set to false cleaning up of the allocated off-heap memory" + + " happens only when the configured JVM parameter MaxDirectMemorySize is reached by triggering a full" + + " GC. For streaming setups is is highly recommended to set this value to false as the core state" + + " backends currently do not use the managed memory.", code(MEMORY_OFF_HEAP.key())).build()); // ------------------------------------------------------------------------ // Network Options From a9ec8dd46e7c5f993a4cc46b18aab974871e93a1 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 25 Feb 2019 15:52:53 +0100 Subject: [PATCH 335/359] [hotfix] Repair ineffective LocalRecoveryITCase The test did not actually run since the class was refactored with JUnit's parameterized, because it was always running into a NPE and the NPE was then silently swallowed in a shutdown catch-block. --- .../apache/flink/test/checkpointing/LocalRecoveryITCase.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java index 53747659d320f..a8626f34e33d6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java @@ -87,6 +87,7 @@ protected Configuration createClusterConfig() throws IOException { private void executeTest(EventTimeWindowCheckpointingITCase delegate) throws Exception { delegate.name = testName; + delegate.stateBackendEnum = backendEnum; try { delegate.setupTestCluster(); try { @@ -94,6 +95,7 @@ private void executeTest(EventTimeWindowCheckpointingITCase delegate) throws Exc delegate.stopTestCluster(); } catch (Exception e) { delegate.stopTestCluster(); + throw new RuntimeException(e); } delegate.setupTestCluster(); @@ -102,9 +104,10 @@ private void executeTest(EventTimeWindowCheckpointingITCase delegate) throws Exc delegate.stopTestCluster(); } catch (Exception e) { delegate.stopTestCluster(); + throw new RuntimeException(e); } } finally { - delegate.tempFolder.delete(); + EventTimeWindowCheckpointingITCase.tempFolder.delete(); } } } From 215204ca2298eb6c2d71baef5d3ea1dce17d368c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 7 Mar 2019 11:41:07 +0100 Subject: [PATCH 336/359] [FLINK-11851] Introduce dedicated io executor for ClusterEntrypoint and MiniCluster The io executor is responsible for running io operations like discarding checkpoints. By using the io executor, we don't risk that the RpcService is blocked by blocking io operations. This closes #7926. --- .../generated/cluster_configuration.html | 16 +++++++++ .../flink/configuration/ClusterOptions.java | 33 +++++++++++++++++++ .../runtime/entrypoint/ClusterEntrypoint.java | 20 ++++++++++- .../runtime/minicluster/MiniCluster.java | 32 ++++++++++++++++-- 4 files changed, 98 insertions(+), 3 deletions(-) create mode 100644 docs/_includes/generated/cluster_configuration.html create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java diff --git a/docs/_includes/generated/cluster_configuration.html b/docs/_includes/generated/cluster_configuration.html new file mode 100644 index 0000000000000..c45a6dd1bed84 --- /dev/null +++ b/docs/_includes/generated/cluster_configuration.html @@ -0,0 +1,16 @@ + + + + + + + + + + + + + + + +
    KeyDefaultDescription
    cluster.services.shutdown-timeout
    30000The shutdown timeout for cluster services like executors in milliseconds.
    diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java new file mode 100644 index 0000000000000..e67dd073aa86a --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Options which control the cluster behaviour. + */ +@PublicEvolving +public class ClusterOptions { + + public static final ConfigOption CLUSTER_SERVICES_SHUTDOWN_TIMEOUT = ConfigOptions + .key("cluster.services.shutdown-timeout") + .defaultValue(30000L) + .withDescription("The shutdown timeout for cluster services like executors in milliseconds."); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 3942d763ae476..48b7120f1c271 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.entrypoint; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; @@ -53,10 +54,13 @@ import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.util.Hardware; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; @@ -80,6 +84,8 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -129,6 +135,9 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro @GuardedBy("lock") private RpcService commonRpcService; + @GuardedBy("lock") + private ExecutorService ioExecutor; + @GuardedBy("lock") private ActorSystem metricQueryServiceActorSystem; @@ -258,7 +267,10 @@ protected void initializeServices(Configuration configuration) throws Exception configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort()); - haServices = createHaServices(configuration, commonRpcService.getExecutor()); + ioExecutor = Executors.newFixedThreadPool( + Hardware.getNumberCPUCores(), + new ExecutorThreadFactory("cluster-io")); + haServices = createHaServices(configuration, ioExecutor); blobServer = new BlobServer(configuration, haServices.createBlobStore()); blobServer.start(); heartbeatServices = createHeartbeatServices(configuration); @@ -324,6 +336,8 @@ public CompletableFuture closeAsync() { } protected CompletableFuture stopClusterServices(boolean cleanupHaData) { + final long shutdownTimeout = configuration.getLong(ClusterOptions.CLUSTER_SERVICES_SHUTDOWN_TIMEOUT); + synchronized (lock) { Throwable exception = null; @@ -373,6 +387,10 @@ protected CompletableFuture stopClusterServices(boolean cleanupHaData) { terminationFutures.add(AkkaUtils.terminateActorSystem(metricQueryServiceActorSystem)); } + if (ioExecutor != null) { + terminationFutures.add(ExecutorUtils.nonBlockingShutdown(shutdownTimeout, TimeUnit.MILLISECONDS, ioExecutor)); + } + if (commonRpcService != null) { terminationFutures.add(commonRpcService.stopService()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 758a9cd8c20ba..f272361b2520b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.RestOptions; @@ -71,11 +72,14 @@ import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.runtime.util.Hardware; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import akka.actor.ActorSystem; @@ -97,6 +101,9 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -134,6 +141,9 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { @GuardedBy("lock") private RpcService resourceManagerRpcService; + @GuardedBy("lock") + private ExecutorService ioExecutor; + @GuardedBy("lock") private ActorSystem metricQueryServiceActorSystem; @@ -296,9 +306,12 @@ public void start() throws Exception { // create the high-availability services LOG.info("Starting high-availability services"); + ioExecutor = Executors.newFixedThreadPool( + Hardware.getNumberCPUCores(), + new ExecutorThreadFactory("mini-cluster-io")); haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices( configuration, - commonRpcService.getExecutor()); + ioExecutor); blobServer = new BlobServer(configuration, haServices.createBlobStore()); blobServer.start(); @@ -432,6 +445,7 @@ public CompletableFuture closeAsync() { if (running) { LOG.info("Shutting down Flink Mini Cluster"); try { + final long shutdownTimeoutMillis = miniClusterConfiguration.getConfiguration().getLong(ClusterOptions.CLUSTER_SERVICES_SHUTDOWN_TIMEOUT); final int numComponents = 2 + miniClusterConfiguration.getNumTaskManagers(); final Collection> componentTerminationFutures = new ArrayList<>(numComponents); @@ -466,7 +480,11 @@ public CompletableFuture closeAsync() { rpcServicesTerminationFuture, this::terminateMiniClusterServices); - remainingServicesTerminationFuture.whenComplete( + final CompletableFuture executorsTerminationFuture = FutureUtils.runAfterwards( + remainingServicesTerminationFuture, + () -> terminateExecutors(shutdownTimeoutMillis)); + + executorsTerminationFuture.whenComplete( (Void ignored, Throwable throwable) -> { if (throwable != null) { terminationFuture.completeExceptionally(ExceptionUtils.stripCompletionException(throwable)); @@ -946,6 +964,16 @@ private CompletionStage terminateRpcServices() { return FutureUtils.completeAll(rpcTerminationFutures); } + private CompletableFuture terminateExecutors(long executorShutdownTimeoutMillis) { + synchronized (lock) { + if (ioExecutor != null) { + return ExecutorUtils.nonBlockingShutdown(executorShutdownTimeoutMillis, TimeUnit.MILLISECONDS, ioExecutor); + } else { + return CompletableFuture.completedFuture(null); + } + } + } + // ------------------------------------------------------------------------ // miscellaneous utilities // ------------------------------------------------------------------------ From a93381360c06a67cd8dca16e3fb5df228c7a63bf Mon Sep 17 00:00:00 2001 From: Tom Goong Date: Mon, 11 Mar 2019 22:24:46 +0800 Subject: [PATCH 337/359] [FLINK-11867][datastream] Fixed preconditions for filePath's value --- .../api/environment/StreamExecutionEnvironment.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index ed69ed4ab8467..9303166dbf943 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -77,6 +77,7 @@ import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SplittableIterator; +import org.apache.flink.util.StringUtils; import com.esotericsoftware.kryo.Serializer; @@ -958,8 +959,7 @@ public DataStreamSource readTextFile(String filePath) { * @return The data stream that represents the data read from the given file as text lines */ public DataStreamSource readTextFile(String filePath, String charsetName) { - Preconditions.checkNotNull(filePath, "The file path must not be null."); - Preconditions.checkNotNull(filePath.isEmpty(), "The file path must not be empty."); + Preconditions.checkArgument(!StringUtils.isNullOrWhitespaceOnly(filePath), "The file path must not be null or blank."); TextInputFormat format = new TextInputFormat(new Path(filePath)); format.setFilesFilter(FilePathFilter.createDefaultFilter()); @@ -1156,8 +1156,7 @@ public DataStreamSource readFile(FileInputFormat inputFormat, TypeInformation typeInformation) { Preconditions.checkNotNull(inputFormat, "InputFormat must not be null."); - Preconditions.checkNotNull(filePath, "The file path must not be null."); - Preconditions.checkNotNull(filePath.isEmpty(), "The file path must not be empty."); + Preconditions.checkArgument(!StringUtils.isNullOrWhitespaceOnly(filePath), "The file path must not be null or blank."); inputFormat.setFilePath(filePath); return createFileInput(inputFormat, typeInformation, "Custom File Source", watchType, interval); From f04d2cb87775b42aa54161ffb3bdaeb1f9d4af3c Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 11 Mar 2019 12:21:29 +0100 Subject: [PATCH 338/359] [FLINK-11420][datastream] Fix duplicate and createInstance methods of CoGroupedStreams.UnionSerializer UnionSerializer did not perform a proper duplication of inner serializers. It also violated the assumption that createInstance never produces null. --- .../api/datastream/CoGroupedStreams.java | 18 ++++- .../api/datastream/UnionSerializerTest.java | 81 +++++++++++++++++++ 2 files changed, 96 insertions(+), 3 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java index 19d978322f09c..047d0b887b3d4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java @@ -504,7 +504,9 @@ public boolean canEqual(Object obj) { } } - private static class UnionSerializer extends TypeSerializer> { + @VisibleForTesting + @Internal + static class UnionSerializer extends TypeSerializer> { private static final long serialVersionUID = 1L; private final TypeSerializer oneSerializer; @@ -522,12 +524,22 @@ public boolean isImmutableType() { @Override public TypeSerializer> duplicate() { - return this; + TypeSerializer duplicateOne = oneSerializer.duplicate(); + TypeSerializer duplicateTwo = twoSerializer.duplicate(); + + // compare reference of nested serializers, if same instances returned, we can reuse + // this instance as well + if (duplicateOne != oneSerializer || duplicateTwo != twoSerializer) { + return new UnionSerializer<>(duplicateOne, duplicateTwo); + } else { + return this; + } } @Override public TaggedUnion createInstance() { - return null; + //we arbitrarily always create instance of one + return TaggedUnion.one(oneSerializer.createInstance()); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerTest.java new file mode 100644 index 0000000000000..29304bcfc8770 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/UnionSerializerTest.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.streaming.api.datastream; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion; +import org.apache.flink.streaming.api.datastream.CoGroupedStreams.UnionSerializer; +import org.apache.flink.testutils.DeeplyEqualsChecker; + +/** + * Serializer tests for {@link UnionSerializer}. + */ +public class UnionSerializerTest extends SerializerTestBase> { + + public UnionSerializerTest() { + super(new DeeplyEqualsChecker() + .withCustomCheck( + (o1, o2) -> o1 instanceof TaggedUnion && o2 instanceof TaggedUnion, + (o1, o2, checker) -> { + TaggedUnion union1 = (TaggedUnion) o1; + TaggedUnion union2 = (TaggedUnion) o2; + + if (union1.isOne() && union2.isOne()) { + return checker.deepEquals(union1.getOne(), union2.getOne()); + } else if (union1.isTwo() && union2.isTwo()) { + return checker.deepEquals(union1.getTwo(), union2.getTwo()); + } else { + return false; + } + } + )); + } + + @Override + protected TypeSerializer> createSerializer() { + return new UnionSerializer<>( + new KryoSerializer<>(Object.class, new ExecutionConfig()), + new KryoSerializer<>(Object.class, new ExecutionConfig()) + ); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + @SuppressWarnings("unchecked") + protected Class> getTypeClass() { + return (Class>) (Class) TaggedUnion.class; + } + + @Override + @SuppressWarnings("unchecked") + protected TaggedUnion[] getTestData() { + return new TaggedUnion[]{ + TaggedUnion.one(1), + TaggedUnion.two("A"), + TaggedUnion.one("C") + }; + } +} From ad1e9fe01234179c2f2e33c22662346a501e8c5d Mon Sep 17 00:00:00 2001 From: chummyhe89 <244272114@qq.com> Date: Wed, 13 Mar 2019 17:06:17 +0800 Subject: [PATCH 339/359] [FLINK-11886][docs] Update CLI output --- docs/ops/jobmanager_high_availability.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ops/jobmanager_high_availability.md b/docs/ops/jobmanager_high_availability.md index 2c2f1a6ba19ed..f37115c9e19fa 100644 --- a/docs/ops/jobmanager_high_availability.md +++ b/docs/ops/jobmanager_high_availability.md @@ -133,17 +133,17 @@ Starting zookeeper daemon on host localhost.
     $ bin/start-cluster.sh
     Starting HA cluster with 2 masters and 1 peers in ZooKeeper quorum.
    -Starting jobmanager daemon on host localhost.
    -Starting jobmanager daemon on host localhost.
    -Starting taskmanager daemon on host localhost.
    +Starting standalonesession daemon on host localhost. +Starting standalonesession daemon on host localhost. +Starting taskexecutor daemon on host localhost. 6. **Stop ZooKeeper quorum and cluster**:
     $ bin/stop-cluster.sh
    -Stopping taskmanager daemon (pid: 7647) on localhost.
    -Stopping jobmanager daemon (pid: 7495) on host localhost.
    -Stopping jobmanager daemon (pid: 7349) on host localhost.
    +Stopping taskexecutor daemon (pid: 7647) on localhost.
    +Stopping standalonesession daemon (pid: 7495) on host localhost.
    +Stopping standalonesession daemon (pid: 7349) on host localhost.
     $ bin/stop-zookeeper-quorum.sh
     Stopping zookeeper daemon (pid: 7101) on host localhost.
    From c3488dde657816c453a7874c26897f6e7cff05e7 Mon Sep 17 00:00:00 2001 From: Romano Vacca Date: Wed, 13 Mar 2019 10:12:55 +0100 Subject: [PATCH 340/359] [FLINK-11866][py][docs] Fix method name --- docs/dev/batch/python.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/batch/python.md b/docs/dev/batch/python.md index 486aa18c9b768..d36d7b15c91ef 100644 --- a/docs/dev/batch/python.md +++ b/docs/dev/batch/python.md @@ -435,7 +435,7 @@ class MyDeserializer(object): return MyObj(i) -env.register_custom_type(MyObj, MySerializer(), MyDeserializer()) +env.register_type(MyObj, MySerializer(), MyDeserializer()) {% endhighlight %} #### Tuples/Lists From fcb4f6a23bd289f8f8db764bc4b9b39949c40359 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 7 Mar 2019 11:44:53 +0100 Subject: [PATCH 341/359] [FLINK-11183][metrics] Move memory metrics creation into separate method --- .../flink/runtime/metrics/MetricNames.java | 4 +++ .../runtime/metrics/util/MetricUtils.java | 34 +++++++++++-------- 2 files changed, 24 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java index dfa82f153816a..5953dc7353144 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java @@ -57,4 +57,8 @@ private MetricNames() { public static final String TASK_SLOTS_AVAILABLE = "taskSlotsAvailable"; public static final String TASK_SLOTS_TOTAL = "taskSlotsTotal"; public static final String NUM_REGISTERED_TASK_MANAGERS = "numRegisteredTaskManagers"; + + public static final String MEMORY_USED = "Used"; + public static final String MEMORY_COMMITTED = "Committed"; + public static final String MEMORY_MAX = "Max"; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java index 6e6f49236b285..0f0629c6f3681 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; @@ -62,6 +63,9 @@ public class MetricUtils { private static final String METRIC_GROUP_STATUS_NAME = "Status"; private static final String METRICS_ACTOR_SYSTEM_NAME = "flink-metrics"; + static final String METRIC_GROUP_HEAP_NAME = "Heap"; + static final String METRIC_GROUP_NONHEAP_NAME = "NonHeap"; + private MetricUtils() { } @@ -159,20 +163,8 @@ private static void instantiateGarbageCollectorMetrics(MetricGroup metrics) { } private static void instantiateMemoryMetrics(MetricGroup metrics) { - final MemoryUsage heapMemoryUsage = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage(); - final MemoryUsage nonHeapMemoryUsage = ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage(); - - MetricGroup heap = metrics.addGroup("Heap"); - - heap.>gauge("Used", heapMemoryUsage::getUsed); - heap.>gauge("Committed", heapMemoryUsage::getCommitted); - heap.>gauge("Max", heapMemoryUsage::getMax); - - MetricGroup nonHeap = metrics.addGroup("NonHeap"); - - nonHeap.>gauge("Used", nonHeapMemoryUsage::getUsed); - nonHeap.>gauge("Committed", nonHeapMemoryUsage::getCommitted); - nonHeap.>gauge("Max", nonHeapMemoryUsage::getMax); + instantiateHeapMemoryMetrics(metrics.addGroup(METRIC_GROUP_HEAP_NAME)); + instantiateNonHeapMemoryMetrics(metrics.addGroup(METRIC_GROUP_NONHEAP_NAME)); final MBeanServer con = ManagementFactory.getPlatformMBeanServer(); @@ -205,6 +197,20 @@ private static void instantiateMemoryMetrics(MetricGroup metrics) { } } + private static void instantiateHeapMemoryMetrics(final MetricGroup metricGroup) { + instantiateMemoryUsageMetrics(metricGroup, ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()); + } + + private static void instantiateNonHeapMemoryMetrics(final MetricGroup metricGroup) { + instantiateMemoryUsageMetrics(metricGroup, ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage()); + } + + private static void instantiateMemoryUsageMetrics(final MetricGroup metricGroup, final MemoryUsage memoryUsage) { + metricGroup.>gauge(MetricNames.MEMORY_USED, memoryUsage::getUsed); + metricGroup.>gauge(MetricNames.MEMORY_COMMITTED, memoryUsage::getCommitted); + metricGroup.>gauge(MetricNames.MEMORY_MAX, memoryUsage::getMax); + } + private static void instantiateThreadMetrics(MetricGroup metrics) { final ThreadMXBean mxBean = ManagementFactory.getThreadMXBean(); From e10a98f56ed89d1a50c0a0c9cb6fc5ba0d8f190f Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 27 Feb 2019 14:56:41 +0100 Subject: [PATCH 342/359] [FLINK-11183][metrics] Properly measure current memory usage --- .../runtime/metrics/util/MetricUtils.java | 20 ++++--- .../runtime/metrics/util/MetricUtilsTest.java | 55 +++++++++++++++++++ 2 files changed, 67 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java index 0f0629c6f3681..a0bbace5f8f38 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.metrics.util; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; @@ -52,6 +53,7 @@ import java.lang.management.ThreadMXBean; import java.util.List; import java.util.Optional; +import java.util.function.Supplier; import static org.apache.flink.runtime.metrics.util.SystemResourcesMetricsInitializer.instantiateSystemMetrics; @@ -197,18 +199,20 @@ private static void instantiateMemoryMetrics(MetricGroup metrics) { } } - private static void instantiateHeapMemoryMetrics(final MetricGroup metricGroup) { - instantiateMemoryUsageMetrics(metricGroup, ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()); + @VisibleForTesting + static void instantiateHeapMemoryMetrics(final MetricGroup metricGroup) { + instantiateMemoryUsageMetrics(metricGroup, () -> ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()); } - private static void instantiateNonHeapMemoryMetrics(final MetricGroup metricGroup) { - instantiateMemoryUsageMetrics(metricGroup, ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage()); + @VisibleForTesting + static void instantiateNonHeapMemoryMetrics(final MetricGroup metricGroup) { + instantiateMemoryUsageMetrics(metricGroup, () -> ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage()); } - private static void instantiateMemoryUsageMetrics(final MetricGroup metricGroup, final MemoryUsage memoryUsage) { - metricGroup.>gauge(MetricNames.MEMORY_USED, memoryUsage::getUsed); - metricGroup.>gauge(MetricNames.MEMORY_COMMITTED, memoryUsage::getCommitted); - metricGroup.>gauge(MetricNames.MEMORY_MAX, memoryUsage::getMax); + private static void instantiateMemoryUsageMetrics(final MetricGroup metricGroup, final Supplier memoryUsageSupplier) { + metricGroup.>gauge(MetricNames.MEMORY_USED, () -> memoryUsageSupplier.get().getUsed()); + metricGroup.>gauge(MetricNames.MEMORY_COMMITTED, () -> memoryUsageSupplier.get().getCommitted()); + metricGroup.>gauge(MetricNames.MEMORY_MAX, () -> memoryUsageSupplier.get().getMax()); } private static void instantiateThreadMetrics(MetricGroup metrics) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java index bea0f2a2153f5..3edb711c351a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/util/MetricUtilsTest.java @@ -20,10 +20,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.util.TestLogger; import akka.actor.ActorSystem; +import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,4 +60,56 @@ public void testStartMetricActorSystemRespectsThreadPriority() throws Exception AkkaUtils.terminateActorSystem(actorSystem).get(); } } + + @Test + public void testNonHeapMetricsCompleteness() { + final InterceptingOperatorMetricGroup nonHeapMetrics = new InterceptingOperatorMetricGroup(); + + MetricUtils.instantiateNonHeapMemoryMetrics(nonHeapMetrics); + + Assert.assertNotNull(nonHeapMetrics.get(MetricNames.MEMORY_USED)); + Assert.assertNotNull(nonHeapMetrics.get(MetricNames.MEMORY_COMMITTED)); + Assert.assertNotNull(nonHeapMetrics.get(MetricNames.MEMORY_MAX)); + } + + @Test + public void testHeapMetricsCompleteness() { + final InterceptingOperatorMetricGroup heapMetrics = new InterceptingOperatorMetricGroup(); + + MetricUtils.instantiateHeapMemoryMetrics(heapMetrics); + + Assert.assertNotNull(heapMetrics.get(MetricNames.MEMORY_USED)); + Assert.assertNotNull(heapMetrics.get(MetricNames.MEMORY_COMMITTED)); + Assert.assertNotNull(heapMetrics.get(MetricNames.MEMORY_MAX)); + } + + /** + * Tests that heap/non-heap metrics do not rely on a static MemoryUsage instance. + * + *

    We can only check this easily for the currently used heap memory, so we use it this as a proxy for testing + * the functionality in general. + */ + @Test + public void testHeapMetrics() throws Exception { + final InterceptingOperatorMetricGroup heapMetrics = new InterceptingOperatorMetricGroup(); + + MetricUtils.instantiateHeapMemoryMetrics(heapMetrics); + + @SuppressWarnings("unchecked") + final Gauge used = (Gauge) heapMetrics.get(MetricNames.MEMORY_USED); + + final long usedHeapInitially = used.getValue(); + + // check memory usage difference multiple times since other tests may affect memory usage as well + for (int x = 0; x < 10; x++) { + final byte[] array = new byte[1024 * 1024 * 8]; + final long usedHeapAfterAllocation = used.getValue(); + + if (usedHeapInitially != usedHeapAfterAllocation) { + return; + } + Thread.sleep(50); + } + Assert.fail("Heap usage metric never changed it's value."); + } } From 17aaca4b6a6a9b0cee37a6d692b5ea9a1b9e9ef2 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 13 Mar 2019 14:08:07 +0100 Subject: [PATCH 343/359] [hotfix][rest] Remove "Impl. error" from log message Original intent was to never reach this code path except on programmer errors, but it has turned into an accepted code path for unhandled exceptions. --- .../org/apache/flink/runtime/rest/handler/AbstractHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java index d46bb13d4265d..dc36aec47597e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java @@ -194,7 +194,7 @@ private CompletableFuture handleException(Throwable throwable, ChannelHand rhe.getHttpResponseStatus(), responseHeaders); } else { - log.error("Implementation error: Unhandled exception.", throwable); + log.error("Unhandled exception.", throwable); String stackTrace = String.format("", ExceptionUtils.stringifyException(throwable)); return HandlerUtils.sendErrorResponse( From a35bfef06f359cb369372881016c9777ff5dbda6 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 13 Mar 2019 14:11:46 +0100 Subject: [PATCH 344/359] [FLINK-11902][rest] Do not wrap all exceptions in RestHandlerException --- .../runtime/webmonitor/handlers/JarRunHandler.java | 10 +--------- .../runtime/webmonitor/handlers/JarRunHandlerTest.java | 4 ++++ 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java index 679dfe988ef5b..46baa2696505d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java @@ -34,8 +34,6 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.FlinkException; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; - import javax.annotation.Nonnull; import java.net.InetSocketAddress; @@ -108,13 +106,7 @@ protected CompletableFuture handleRequest( }); return jobSubmissionFuture - .thenCombine(jarUploadFuture, (ack, jobGraph) -> new JarRunResponseBody(jobGraph.getJobID())) - .exceptionally(throwable -> { - throw new CompletionException(new RestHandlerException( - throwable.getMessage(), - HttpResponseStatus.INTERNAL_SERVER_ERROR, - throwable)); - }); + .thenCombine(jarUploadFuture, (ack, jobGraph) -> new JarRunResponseBody(jobGraph.getJobID())); } private SavepointRestoreSettings getSavepointRestoreSettings( diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java index 5fc3ff5c06790..1b236357ed589 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java @@ -39,6 +39,8 @@ import java.nio.file.Path; import java.util.Optional; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -90,6 +92,8 @@ public void testRunJar() throws Exception { if (expected.isPresent()) { // implies the job was actually submitted assertTrue(expected.get().getMessage().contains("ProgramInvocationException")); + // original cause is preserved in stack trace + assertThat(expected.get().getMessage(), containsString("ZipException")); // implies the jar was registered for the job graph (otherwise the jar name would not occur in the exception) // implies the jar was uploaded (otherwise the file would not be found at all) assertTrue(expected.get().getMessage().contains("empty.jar'. zip file is empty")); From 5ef14a151255d0d758f2a0a0086a02622ce4f801 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 1 Mar 2019 08:45:11 +0100 Subject: [PATCH 345/359] [FLINK-11786][travis] Simplify stage selection --- .travis.yml | 15 ++++++------- tools/travis_controller.sh | 44 +++++--------------------------------- 2 files changed, 12 insertions(+), 47 deletions(-) diff --git a/.travis.yml b/.travis.yml index 49b4527427e76..5a85c36ad82f8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -60,32 +60,31 @@ before_install: - chmod +x docker-compose - sudo mv docker-compose /usr/local/bin -# When modifying the matrix you also have to modify travis_controller.sh#getCurrentStage jdk: "oraclejdk8" jobs: include: # main profile - stage: compile - script: ./tools/travis_controller.sh + script: ./tools/travis_controller.sh compile env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: compile - stage: test - script: ./tools/travis_controller.sh + script: ./tools/travis_controller.sh core env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: core - - script: ./tools/travis_controller.sh + - script: ./tools/travis_controller.sh libraries env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: libraries - - script: ./tools/travis_controller.sh + - script: ./tools/travis_controller.sh connectors env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: connectors - - script: ./tools/travis_controller.sh + - script: ./tools/travis_controller.sh tests env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: tests - - script: ./tools/travis_controller.sh + - script: ./tools/travis_controller.sh misc env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: misc - stage: cleanup - script: ./tools/travis_controller.sh + script: ./tools/travis_controller.sh cleanup env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: cleanup diff --git a/tools/travis_controller.sh b/tools/travis_controller.sh index 4868057c6cfd0..39ada67a0b607 100755 --- a/tools/travis_controller.sh +++ b/tools/travis_controller.sh @@ -52,44 +52,7 @@ function deleteOldCaches() { # delete leftover caches from previous builds find "$CACHE_DIR" -mindepth 1 -maxdepth 1 | grep -v "$TRAVIS_BUILD_NUMBER" | deleteOldCaches -function getCurrentStage() { - STAGE_NUMBER=$(echo "$TRAVIS_JOB_NUMBER" | cut -d'.' -f 2) - case $STAGE_NUMBER in - (1) - echo "$STAGE_COMPILE" - ;; - (2) - echo "$STAGE_CORE" - ;; - (3) - echo "$STAGE_LIBRARIES" - ;; - (4) - echo "$STAGE_CONNECTORS" - ;; - (5) - echo "$STAGE_TESTS" - ;; - (6) - echo "$STAGE_MISC" - ;; - (7) - echo "$STAGE_CLEANUP" - ;; - (*) - echo "Invalid stage detected ($STAGE_NUMBER)" - return 1 - ;; - esac - - return 0 -} - -STAGE=$(getCurrentStage) -if [ $? != 0 ]; then - echo "Could not determine current stage." - exit 1 -fi +STAGE=$1 echo "Current stage: \"$STAGE\"" EXIT_CODE=0 @@ -200,9 +163,12 @@ elif [ $STAGE != "$STAGE_CLEANUP" ]; then TEST="$STAGE" "./tools/travis_mvn_watchdog.sh" 300 EXIT_CODE=$? -else +elif [ $STAGE == "$STAGE_CLEANUP" ]; then echo "Cleaning up $CACHE_BUILD_DIR" rm -rf "$CACHE_BUILD_DIR" +else + echo "Invalid Stage specified: $STAGE" + exit 1 fi # Exit code for Travis build success/failure From a933a7c67a9a9a28ce62a147399f75c1bae2c8b6 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 1 Mar 2019 08:46:39 +0100 Subject: [PATCH 346/359] [FLINK-11786][travis] Run main profile only on pr/push --- .travis.yml | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/.travis.yml b/.travis.yml index 5a85c36ad82f8..de09549296639 100644 --- a/.travis.yml +++ b/.travis.yml @@ -64,27 +64,34 @@ jdk: "oraclejdk8" jobs: include: # main profile - - stage: compile + - if: type in (pull_request, push) + stage: compile script: ./tools/travis_controller.sh compile env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: compile - - stage: test + - if: type in (pull_request, push) + stage: test script: ./tools/travis_controller.sh core env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: core - - script: ./tools/travis_controller.sh libraries + - if: type in (pull_request, push) + script: ./tools/travis_controller.sh libraries env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: libraries - - script: ./tools/travis_controller.sh connectors + - if: type in (pull_request, push) + script: ./tools/travis_controller.sh connectors env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: connectors - - script: ./tools/travis_controller.sh tests + - if: type in (pull_request, push) + script: ./tools/travis_controller.sh tests env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: tests - - script: ./tools/travis_controller.sh misc + - if: type in (pull_request, push) + script: ./tools/travis_controller.sh misc env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: misc - - stage: cleanup + - if: type in (pull_request, push) + stage: cleanup script: ./tools/travis_controller.sh cleanup env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: cleanup From 17509d82ec0250e2e491abb86ea3b4941726d174 Mon Sep 17 00:00:00 2001 From: zentol Date: Fri, 1 Mar 2019 08:47:08 +0100 Subject: [PATCH 347/359] [FLINK-11786][travis] Merge cron jobs --- .travis.yml | 28 +++++++ tools/travis/nightly.sh | 82 ++++++++++++++++++++ tools/travis/splits/split_checkpoints.sh | 78 +++++++++++++++++++ tools/travis/splits/split_container.sh | 51 ++++++++++++ tools/travis/splits/split_ha.sh | 59 ++++++++++++++ tools/travis/splits/split_heavy.sh | 51 ++++++++++++ tools/travis/splits/split_misc.sh | 77 ++++++++++++++++++ tools/travis/splits/split_misc_hadoopfree.sh | 71 +++++++++++++++++ tools/travis/splits/split_sticky.sh | 54 +++++++++++++ 9 files changed, 551 insertions(+) create mode 100755 tools/travis/nightly.sh create mode 100755 tools/travis/splits/split_checkpoints.sh create mode 100755 tools/travis/splits/split_container.sh create mode 100755 tools/travis/splits/split_ha.sh create mode 100755 tools/travis/splits/split_heavy.sh create mode 100755 tools/travis/splits/split_misc.sh create mode 100755 tools/travis/splits/split_misc_hadoopfree.sh create mode 100755 tools/travis/splits/split_sticky.sh diff --git a/.travis.yml b/.travis.yml index de09549296639..ab5c2018c0a36 100644 --- a/.travis.yml +++ b/.travis.yml @@ -23,6 +23,7 @@ cache: # default timeout is too low timeout: 600 directories: + - $HOME/.rvm/ - $HOME/.m2 - $HOME/flink_cache # keep in sync with tools/travis/setup_maven.sh @@ -60,6 +61,13 @@ before_install: - chmod +x docker-compose - sudo mv docker-compose /usr/local/bin +stages: + - name: compile + - name: test + - name: E2E + if: type = cron + - name: cleanup + jdk: "oraclejdk8" jobs: include: @@ -95,3 +103,23 @@ jobs: script: ./tools/travis_controller.sh cleanup env: PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis -Dinclude_hadoop_aws -Dscala-2.11" name: cleanup + # E2E profile + - stage: E2E + env: PROFILE="-Dhadoop.version=2.8.3 -De2e-metrics" + script: ./tools/travis/nightly.sh split_misc.sh + name: misc - hadoop 2.8 + - env: PROFILE="-Dhadoop.version=2.8.3" + script: ./tools/travis/nightly.sh split_ha.sh + name: ha - hadoop 2.8 + - env: PROFILE="-Dhadoop.version=2.8.3" + script: ./tools/travis/nightly.sh split_sticky.sh + name: sticky - hadoop 2.8 + - env: PROFILE="-Dhadoop.version=2.8.3" + script: ./tools/travis/nightly.sh split_checkpoints.sh + name: checkpoints - hadoop 2.8 + - env: PROFILE="-Dhadoop.version=2.8.3" + script: ./tools/travis/nightly.sh split_container.sh + name: container - hadoop 2.8 + - env: PROFILE="-Dhadoop.version=2.8.3" + script: ./tools/travis/nightly.sh split_heavy.sh + name: heavy - hadoop 2.8 diff --git a/tools/travis/nightly.sh b/tools/travis/nightly.sh new file mode 100755 index 0000000000000..0aad37aa3babb --- /dev/null +++ b/tools/travis/nightly.sh @@ -0,0 +1,82 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +HERE="`dirname \"$0\"`" # relative +HERE="`( cd \"${HERE}\" && pwd -P)`" # absolutized and normalized +if [ -z "${HERE}" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +ARTIFACTS_DIR="${HERE}/artifacts" +FLINK_DIR="${HERE}/flink" + +mkdir -p $ARTIFACTS_DIR || { echo "FAILURE: cannot create log directory '${ARTIFACTS_DIR}'." ; exit 1; } + +LOG4J_PROPERTIES=${FLINK_DIR}/tools/log4j-travis.properties + +MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" +MVN_COMMON_OPTIONS="-nsu -B -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast" +MVN_COMPILE_OPTIONS="-T1C -DskipTests" + +cp tools/travis/splits/* ${FLINK_DIR}/flink-end-to-end-tests + +cd "${FLINK_DIR}" + +COMMIT_HASH=$(git rev-parse HEAD) +echo "Testing branch ${BRANCH} from remote ${REMOTE}. Commit hash: ${COMMIT_HASH}" + +e2e_modules=$(find flink-end-to-end-tests -mindepth 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',') +MVN_COMPILE="mvn ${MVN_COMMON_OPTIONS} ${MVN_COMPILE_OPTIONS} ${MVN_LOGGING_OPTIONS} ${PROFILE} clean install -pl ${e2e_modules},flink-dist -am" + +eval "${MVN_COMPILE}" +EXIT_CODE=$? + +if [ $EXIT_CODE == 0 ]; then + printf "\n\n==============================================================================\n" + printf "Running Java end-to-end tests\n" + printf "==============================================================================\n" + + MVN_TEST="mvn ${MVN_COMMON_OPTIONS} ${MVN_LOGGING_OPTIONS} ${PROFILE} verify -pl ${e2e_modules} -DdistDir=$(readlink -e build-target)" + + eval "${MVN_TEST}" + EXIT_CODE=$? +else + printf "\n\n==============================================================================\n" + printf "Compile failure detected, skipping Java end-to-end tests\n" + printf "==============================================================================\n" +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n\n==============================================================================\n" + printf "Running end-to-end tests\n" + printf "==============================================================================\n" + + FLINK_DIR=build-target flink-end-to-end-tests/${SCRIPT} + + EXIT_CODE=$? +else + printf "\n\n==============================================================================\n" + printf "Compile failure detected, skipping end-to-end tests\n" + printf "==============================================================================\n" +fi + +# Exit code for Travis build success/failure +exit ${EXIT_CODE} diff --git a/tools/travis/splits/split_checkpoints.sh b/tools/travis/splits/split_checkpoints.sh new file mode 100755 index 0000000000000..5dfe7e344027e --- /dev/null +++ b/tools/travis/splits/split_checkpoints.sh @@ -0,0 +1,78 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd )`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true" +run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false" +run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true" +run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false" +run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true" +run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false" +run_test "Resuming Savepoint (rocks, no parallelism change, heap timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks false heap" +run_test "Resuming Savepoint (rocks, scale up, heap timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks false heap" +run_test "Resuming Savepoint (rocks, scale down, heap timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks false heap" +run_test "Resuming Savepoint (rocks, no parallelism change, rocks timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks false rocks" +run_test "Resuming Savepoint (rocks, scale up, rocks timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks false rocks" +run_test "Resuming Savepoint (rocks, scale down, rocks timers) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks false rocks" + +run_test "Resuming Externalized Checkpoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 file true true" +run_test "Resuming Externalized Checkpoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 file false true" +run_test "Resuming Externalized Checkpoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 4 file true true" +run_test "Resuming Externalized Checkpoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 4 file false true" +run_test "Resuming Externalized Checkpoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 4 2 file true true" +run_test "Resuming Externalized Checkpoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 4 2 file false true" +run_test "Resuming Externalized Checkpoint (rocks, non-incremental, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 rocks true false" +run_test "Resuming Externalized Checkpoint (rocks, incremental, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 rocks true true" +run_test "Resuming Externalized Checkpoint (rocks, non-incremental, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 4 rocks true false" +run_test "Resuming Externalized Checkpoint (rocks, incremental, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 4 rocks true true" +run_test "Resuming Externalized Checkpoint (rocks, non-incremental, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 4 2 rocks true false" +run_test "Resuming Externalized Checkpoint (rocks, incremental, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 4 2 rocks true true" + +run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 file true false true" +run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 file false false true" +run_test "Resuming Externalized Checkpoint after terminal failure (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 rocks true false true" +run_test "Resuming Externalized Checkpoint after terminal failure (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh 2 2 rocks true true true" + +printf "\n[PASS] All tests passed\n" +exit 0 diff --git a/tools/travis/splits/split_container.sh b/tools/travis/splits/split_container.sh new file mode 100755 index 0000000000000..c62905f9a1de9 --- /dev/null +++ b/tools/travis/splits/split_container.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd -P)`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd -P)`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Running Kerberized YARN on Docker test " "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" + +run_test "Run kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" + +printf "\n[PASS] All tests passed\n" +exit 0 diff --git a/tools/travis/splits/split_ha.sh b/tools/travis/splits/split_ha.sh new file mode 100755 index 0000000000000..628b716d7cbba --- /dev/null +++ b/tools/travis/splits/split_ha.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd -P)`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd -P)`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions" + +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false" "skip_check_exceptions" +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false" "skip_check_exceptions" +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false" "skip_check_exceptions" +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true" "skip_check_exceptions" + +run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false" "skip_check_exceptions" +run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true" "skip_check_exceptions" + +printf "\n[PASS] All tests passed\n" +exit 0 diff --git a/tools/travis/splits/split_heavy.sh b/tools/travis/splits/split_heavy.sh new file mode 100755 index 0000000000000..e182d733179a4 --- /dev/null +++ b/tools/travis/splits/split_heavy.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd )`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Heavy deployment end-to-end test" "$END_TO_END_DIR/test-scripts/test_heavy_deployment.sh" "skip_check_exceptions" + +run_test "ConnectedComponents iterations with high parallelism end-to-end test" "$END_TO_END_DIR/test-scripts/test_high_parallelism_iterations.sh 25" + +printf "\n[PASS] All tests passed\n" +exit 0 diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh new file mode 100755 index 0000000000000..e98c51cd475c9 --- /dev/null +++ b/tools/travis/splits/split_misc.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd -P)`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd -P)`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Flink CLI end-to-end test" "$END_TO_END_DIR/test-scripts/test_cli.sh" + +run_test "Queryable state (rocksdb) end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state.sh rocksdb" +run_test "Queryable state (rocksdb) with TM restart end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state_restart_tm.sh" "skip_check_exceptions" + +run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh" +run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh" "skip_check_exceptions" +run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh" "skip_check_exceptions" +run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh" "skip_check_exceptions" +run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" +run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" + +run_test "Elasticsearch (v1.7.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz" +run_test "Elasticsearch (v2.3.5) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz" +run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" + +run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java" +run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh scala" + +run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_confluent_schema_registry.sh" + +run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file" +run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh rocks" + +run_test "SQL Client end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_client.sh" +run_test "SQL Client end-to-end test for Kafka 0.10" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka010.sh" +run_test "SQL Client end-to-end test for Kafka 0.11" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka011.sh" +run_test "SQL Client end-to-end test for modern Kafka" "$END_TO_END_DIR/test-scripts/test_sql_client_kafka.sh" + +printf "\n[PASS] All tests passed\n" +exit 0 diff --git a/tools/travis/splits/split_misc_hadoopfree.sh b/tools/travis/splits/split_misc_hadoopfree.sh new file mode 100755 index 0000000000000..6c552bd7940e3 --- /dev/null +++ b/tools/travis/splits/split_misc_hadoopfree.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd -P)`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd -P)`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Flink CLI end-to-end test" "$END_TO_END_DIR/test-scripts/test_cli.sh" + +run_test "Queryable state (rocksdb) end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state.sh rocksdb" +run_test "Queryable state (rocksdb) with TM restart end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state_restart_tm.sh" "skip_check_exceptions" + +run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh" +run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh" "skip_check_exceptions" +run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh" "skip_check_exceptions" +run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" +run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" + +run_test "Elasticsearch (v1.7.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz" +run_test "Elasticsearch (v2.3.5) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz" +run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" + +run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java" +run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh scala" + +run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_confluent_schema_registry.sh" + +run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file" +run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh rocks" + +printf "\n[PASS] All tests passed\n" +exit 0 diff --git a/tools/travis/splits/split_sticky.sh b/tools/travis/splits/split_sticky.sh new file mode 100755 index 0000000000000..010f887163fd7 --- /dev/null +++ b/tools/travis/splits/split_sticky.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd -P)`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +export END_TO_END_DIR + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd -P)`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +# Template for adding a test: + +# run_test "" "$END_TO_END_DIR/test-scripts/" + +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false false" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false true" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false false" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true false" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false true" "skip_check_exceptions" +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true true" "skip_check_exceptions" + +printf "\n[PASS] All tests passed\n" +exit 0 From 5ba2c3f4ef99010c9a68acf1b701a3930b983994 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 6 Mar 2019 14:06:35 +0100 Subject: [PATCH 348/359] [FLINK-11786][travis] Setup notifications --- .travis.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.travis.yml b/.travis.yml index ab5c2018c0a36..5c8e7b1a2b8a9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -61,6 +61,13 @@ before_install: - chmod +x docker-compose - sudo mv docker-compose /usr/local/bin +notifications: + slack: + rooms: + - secure: ikPQn5JTpkyzxVyOPm/jIl3FPm6hY8xAdG4pSwxGWjBqF+NmmNTp9YZsJ6fD8xPql6T5n1hNDbZSC14jVUw/vvXGvibDXLN+06f25ZQl+4LJBXaiR7gTG6y3nO8G90Vw7XpvCme6n5Md9tvjygb17a4FEgRJFfwzWnnyPA1yvK0= + on_success: never + on_pull_requests: false + stages: - name: compile - name: test From 34b4da1c2160dbb07328dfd76d6cb9d00c15b081 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 21 Mar 2019 15:06:32 +0100 Subject: [PATCH 349/359] [hotfix][travis] Fix directory references --- tools/travis/nightly.sh | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/tools/travis/nightly.sh b/tools/travis/nightly.sh index 0aad37aa3babb..1b547ad9ed314 100755 --- a/tools/travis/nightly.sh +++ b/tools/travis/nightly.sh @@ -26,19 +26,16 @@ if [ -z "${HERE}" ] ; then fi ARTIFACTS_DIR="${HERE}/artifacts" -FLINK_DIR="${HERE}/flink" mkdir -p $ARTIFACTS_DIR || { echo "FAILURE: cannot create log directory '${ARTIFACTS_DIR}'." ; exit 1; } -LOG4J_PROPERTIES=${FLINK_DIR}/tools/log4j-travis.properties +LOG4J_PROPERTIES=tools/log4j-travis.properties MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" MVN_COMMON_OPTIONS="-nsu -B -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast" MVN_COMPILE_OPTIONS="-T1C -DskipTests" -cp tools/travis/splits/* ${FLINK_DIR}/flink-end-to-end-tests - -cd "${FLINK_DIR}" +cp tools/travis/splits/* flink-end-to-end-tests COMMIT_HASH=$(git rev-parse HEAD) echo "Testing branch ${BRANCH} from remote ${REMOTE}. Commit hash: ${COMMIT_HASH}" From c655c3b275c927277799dc3ec52dcc7637d6c132 Mon Sep 17 00:00:00 2001 From: SuXingLee <913910636@qq.com> Date: Fri, 22 Mar 2019 18:36:55 +0800 Subject: [PATCH 350/359] [FLINK-11887][metrics] Fix latency drift --- .../flink/streaming/api/operators/StreamSource.java | 2 +- .../StreamSourceOperatorLatencyMetricsTest.java | 10 +++++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java index 63dd3e4d42740..f09d83e45119b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java @@ -155,7 +155,7 @@ public LatencyMarksEmitter( public void onProcessingTime(long timestamp) throws Exception { try { // ProcessingTimeService callbacks are executed under the checkpointing lock - output.emitLatencyMarker(new LatencyMarker(timestamp, operatorId, subtaskIndex)); + output.emitLatencyMarker(new LatencyMarker(processingTimeService.getCurrentProcessingTime(), operatorId, subtaskIndex)); } catch (Throwable t) { // we catch the Throwables here so that we don't trigger the processing // timer services async exception handler diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java index 14d51474b5753..dbcfc0df7d310 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java @@ -175,6 +175,7 @@ private void testLatencyMarkEmission(int numberLatencyMarkers, OperatorSetupOper output.size()); long timestamp = 0L; + int expectedLatencyIndex = 0; int i = 0; // verify that its only latency markers + a final watermark @@ -183,7 +184,14 @@ private void testLatencyMarkEmission(int numberLatencyMarkers, OperatorSetupOper Assert.assertTrue(se.isLatencyMarker()); Assert.assertEquals(operator.getOperatorID(), se.asLatencyMarker().getOperatorId()); Assert.assertEquals(0, se.asLatencyMarker().getSubtaskIndex()); - Assert.assertTrue(se.asLatencyMarker().getMarkedTime() == timestamp); + + // determines the next latency mark that should've been emitted + // latency marks are emitted once per latencyMarkInterval, + // as a result of which we never emit both 10 and 11 + while (timestamp > processingTimes.get(expectedLatencyIndex)) { + expectedLatencyIndex++; + } + Assert.assertEquals(processingTimes.get(expectedLatencyIndex).longValue(), se.asLatencyMarker().getMarkedTime()); timestamp += latencyMarkInterval; } From 8321f64401e13b203bf74cc4dc77a7e4bdd69d5d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 22 Mar 2019 12:29:10 +0100 Subject: [PATCH 351/359] [hotfix][travis] Fix e2e setup issues --- .travis.yml | 10 ++++++++++ tools/travis/nightly.sh | 2 ++ 2 files changed, 12 insertions(+) diff --git a/.travis.yml b/.travis.yml index 5c8e7b1a2b8a9..19922a1bea19e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -45,12 +45,22 @@ env: # Global variable to avoid hanging travis builds when downloading cache archives. - MALLOC_ARENA_MAX=2 - DOCKER_COMPOSE_VERSION=1.22.0 + - CHANGE_MINIKUBE_NONE_USER=true before_script: - "gem install --no-document --version 0.8.9 faraday " - "export -f travis_nanoseconds" - "export -f travis_time_start" - "export -f travis_time_finish" + # Download kubectl, which is a requirement for using minikube. + - curl -Lo kubectl https://storage.googleapis.com/kubernetes-release/release/$(curl -s https://storage.googleapis.com/kubernetes-release/release/stable.txt)/bin/linux/amd64/kubectl && chmod +x kubectl && sudo mv kubectl /usr/local/bin/ + # Download minikube. + - curl -Lo minikube https://storage.googleapis.com/minikube/releases/v0.25.2/minikube-linux-amd64 && chmod +x minikube && sudo mv minikube /usr/local/bin/ + - sudo minikube start --vm-driver=none --kubernetes-version=v1.9.0 + # Fix the kubectl context, as it's often stale. + - minikube update-context + # Wait for Kubernetes to be up and ready. + - JSONPATH='{range .items[*]}{@.metadata.name}:{range @.status.conditions[*]}{@.type}={@.status};{end}{end}'; until kubectl get nodes -o jsonpath="$JSONPATH" 2>&1 | grep -q "Ready=True"; do sleep 1; done # Install maven 3.2.5 since trusty uses 3.3.9 for which shading is broken before_install: diff --git a/tools/travis/nightly.sh b/tools/travis/nightly.sh index 1b547ad9ed314..c7429316054e3 100755 --- a/tools/travis/nightly.sh +++ b/tools/travis/nightly.sh @@ -25,6 +25,8 @@ if [ -z "${HERE}" ] ; then exit 1 # fail fi +SCRIPT=$1 + ARTIFACTS_DIR="${HERE}/artifacts" mkdir -p $ARTIFACTS_DIR || { echo "FAILURE: cannot create log directory '${ARTIFACTS_DIR}'." ; exit 1; } From 11c3e6a8518673376187f06ab0773e0e8ca6a778 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 20 Mar 2019 13:53:07 +0100 Subject: [PATCH 352/359] [FLINK-11984][docs] MPU timeout implications on StreamingFileSink. --- docs/dev/connectors/streamfile_sink.md | 28 ++++++++++++++++++-------- 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index 82ab5620571ba..353a1f84c2aaa 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -26,14 +26,6 @@ under the License. This connector provides a Sink that writes partitioned files to filesystems supported by the [Flink `FileSystem` abstraction]({{ site.baseurl}}/ops/filesystems.html). -Important Note: For S3, the `StreamingFileSink` -supports only the [Hadoop-based](https://hadoop.apache.org/) FileSystem implementation, not -the implementation based on [Presto](https://prestodb.io/). In case your job uses the -`StreamingFileSink` to write to S3 but you want to use the Presto-based one for checkpointing, -it is advised to use explicitly *"s3a://"* (for Hadoop) as the scheme for the target path of -the sink and *"s3p://"* for checkpointing (for Presto). Using *"s3://"* for both the sink -and checkpointing may lead to unpredictable behavior, as both implementations "listen" to that scheme. - Since in streaming the input is potentially infinite, the streaming file sink writes data into buckets. The bucketing behaviour is configurable but a useful default is time-based bucketing where we start writing a new bucket every hour and thus get @@ -123,4 +115,24 @@ has static methods for creating a `BulkWriter.Factory` for various types. every checkpoint. +#### Important Considerations for S3 + +Important Note 1: For S3, the `StreamingFileSink` +supports only the [Hadoop-based](https://hadoop.apache.org/) FileSystem implementation, not +the implementation based on [Presto](https://prestodb.io/). In case your job uses the +`StreamingFileSink` to write to S3 but you want to use the Presto-based one for checkpointing, +it is advised to use explicitly *"s3a://"* (for Hadoop) as the scheme for the target path of +the sink and *"s3p://"* for checkpointing (for Presto). Using *"s3://"* for both the sink +and checkpointing may lead to unpredictable behavior, as both implementations "listen" to that scheme. + +Important Note 2: To guarantee exactly-once semantics while +being efficient, the `StreamingFileSink` uses the [Multi-part Upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/mpuoverview.html) +feature of S3 (MPU from now on). This feature allows to upload files in independent chunks (thus the "multi-part") +which can be combined into the original file when all the parts of the MPU are successfully uploaded. +For inactive MPUs, S3 supports a bucket lifecycle rule that the user can use to abort multipart uploads +that don't complete within a specified number of days after being initiated. This implies that if you set this rule +aggressively and take a savepoint with some part-files being not fully uploaded, their associated MPUs may time-out +before the job is restarted. This will result in your job not being able to restore from that savepoint as the +pending part-files are no longer there and Flink will fail with an exception as it tries to fetch them and fails. + {% top %} From 565c49a1dddfa48d66abcafe5a846c613afebf4c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 25 Mar 2019 17:59:42 +0100 Subject: [PATCH 353/359] [hotfix] Increase startup timeout in end-to-end tests We've seen quite some flakyness in the end-to-end tests on Travis lately. On most tests it takes about 5-8 secs for the dispatcher to come up so 10 secs might be to low. --- flink-end-to-end-tests/test-scripts/common.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 183735b0a1071..2f4c849960026 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -212,7 +212,7 @@ function start_local_zk { function wait_dispatcher_running { # wait at most 10 seconds until the dispatcher is up local QUERY_URL="${REST_PROTOCOL}://${NODENAME}:8081/taskmanagers" - local TIMEOUT=10 + local TIMEOUT=20 for i in $(seq 1 ${TIMEOUT}); do # without the || true this would exit our script if the JobManager is not yet up QUERY_RESULT=$(curl ${CURL_SSL_ARGS} "$QUERY_URL" 2> /dev/null || true) From 4eb0aebb4832cc0b070afc3bcf0838aebe05578b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 7 Mar 2019 17:28:33 +0100 Subject: [PATCH 354/359] [FLINK-11855] Fix race condition in EmbeddedLeaderService#GrantLeadershipCall Fix the race condition between executing EmbeddedLeaderService#GrantLeadershipCall and a concurrent shutdown of the leader service by making GrantLeadershipCall not accessing mutable state outside of a lock. This closes #7937. --- .../nonha/embedded/EmbeddedLeaderService.java | 37 ++++++-- .../embedded/EmbeddedHaServicesTest.java | 1 + .../embedded/EmbeddedLeaderServiceTest.java | 67 ++++++++++++++ .../embedded/TestingLeaderContender.java | 87 +++++++++++++++++++ 4 files changed, 183 insertions(+), 9 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/TestingLeaderContender.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java index dafefcccef29f..3ca33fcbd62d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderService.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.highavailability.nonha.embedded; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; @@ -96,6 +97,13 @@ public void shutdown() { } } + @VisibleForTesting + public boolean isShutdown() { + synchronized (lock) { + return shutdown; + } + } + private void fatalError(Throwable error) { LOG.error("Embedded leader election service encountered a fatal error. Shutting down service.", error); @@ -270,12 +278,13 @@ private void updateLeader() { currentLeaderSessionId = leaderSessionId; currentLeaderProposed = leaderService; + currentLeaderProposed.isLeader = true; LOG.info("Proposing leadership to contender {} @ {}", leaderService.contender, leaderService.contender.getAddress()); notificationExecutor.execute( - new GrantLeadershipCall(leaderService, leaderSessionId, LOG)); + new GrantLeadershipCall(leaderService.contender, leaderSessionId, LOG)); } } } @@ -440,34 +449,44 @@ public void run() { private static class GrantLeadershipCall implements Runnable { - private final EmbeddedLeaderElectionService leaderElectionService; + private final LeaderContender contender; private final UUID leaderSessionId; private final Logger logger; GrantLeadershipCall( - EmbeddedLeaderElectionService leaderElectionService, + LeaderContender contender, UUID leaderSessionId, Logger logger) { - this.leaderElectionService = checkNotNull(leaderElectionService); + this.contender = checkNotNull(contender); this.leaderSessionId = checkNotNull(leaderSessionId); this.logger = checkNotNull(logger); } @Override public void run() { - leaderElectionService.isLeader = true; - - final LeaderContender contender = leaderElectionService.contender; - try { contender.grantLeadership(leaderSessionId); } catch (Throwable t) { logger.warn("Error granting leadership to contender", t); contender.handleError(t instanceof Exception ? (Exception) t : new Exception(t)); - leaderElectionService.isLeader = false; } } } + + private static class RevokeLeadershipCall implements Runnable { + + @Nonnull + private final LeaderContender contender; + + RevokeLeadershipCall(@Nonnull LeaderContender contender) { + this.contender = contender; + } + + @Override + public void run() { + contender.revokeLeadership(); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java index 7bf9364c8d3b3..9913f36ed6c28 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.util.TestLogger; + import org.junit.After; import org.junit.Before; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java new file mode 100644 index 0000000000000..e4c7cc2bf91a4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java @@ -0,0 +1,67 @@ +/* + * 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.highavailability.nonha.embedded; + +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.hamcrest.Matchers.is; + +/** + * Tests for the {@link EmbeddedLeaderService}. + */ +public class EmbeddedLeaderServiceTest extends TestLogger { + + /** + * Tests that the {@link EmbeddedLeaderService} can handle a concurrent grant + * leadership call and a shutdown. + */ + @Test + public void testConcurrentGrantLeadershipAndShutdown() throws Exception { + final EmbeddedLeaderService embeddedLeaderService = new EmbeddedLeaderService(TestingUtils.defaultExecutor()); + + try { + final LeaderElectionService leaderElectionService = embeddedLeaderService.createLeaderElectionService(); + + final TestingLeaderContender contender = new TestingLeaderContender(); + + leaderElectionService.start(contender); + leaderElectionService.stop(); + + try { + // check that no exception occurred + contender.getLeaderSessionFuture().get(10L, TimeUnit.MILLISECONDS); + } catch (TimeoutException ignored) { + // we haven't participated in the leader election + } + + // the election service should still be running + Assert.assertThat(embeddedLeaderService.isShutdown(), is(false)); + } finally { + embeddedLeaderService.shutdown(); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/TestingLeaderContender.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/TestingLeaderContender.java new file mode 100644 index 0000000000000..0b286716769ce --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/TestingLeaderContender.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.runtime.highavailability.nonha.embedded; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.leaderelection.LeaderContender; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * {@link LeaderContender} implementation for testing purposes. + */ +final class TestingLeaderContender implements LeaderContender { + + private final Object lock = new Object(); + + private CompletableFuture leaderSessionFuture; + + TestingLeaderContender() { + leaderSessionFuture = new CompletableFuture<>(); + } + + @Override + public void grantLeadership(UUID leaderSessionID) { + synchronized (lock) { + if (!leaderSessionFuture.isCompletedExceptionally()) { + if (!leaderSessionFuture.complete(leaderSessionID)) { + leaderSessionFuture = CompletableFuture.completedFuture(leaderSessionID); + } + } + } + } + + @Override + public void revokeLeadership() { + synchronized (lock) { + if (leaderSessionFuture.isDone() && !leaderSessionFuture.isCompletedExceptionally()) { + leaderSessionFuture = new CompletableFuture<>(); + } + } + } + + @Override + public String getAddress() { + return "foobar"; + } + + @Override + public void handleError(Exception exception) { + synchronized (lock) { + if (!(leaderSessionFuture.isCompletedExceptionally() || leaderSessionFuture.completeExceptionally(exception))) { + leaderSessionFuture = FutureUtils.completedExceptionally(exception); + } + } + } + + public void tryRethrowException() { + synchronized (lock) { + if (leaderSessionFuture.isCompletedExceptionally()) { + leaderSessionFuture.getNow(null); + } + } + } + + CompletableFuture getLeaderSessionFuture() { + synchronized (lock) { + return leaderSessionFuture; + } + } +} From 459965f93e40bedcf0b426cc068f333c8243c4ba Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 27 Mar 2019 09:20:12 +0100 Subject: [PATCH 355/359] [hotfix] Fix checkstyle violations in ExecutionGraphDeploymentTest --- .../ExecutionGraphDeploymentTest.java | 30 +++++-------------- 1 file changed, 8 insertions(+), 22 deletions(-) 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 12b4277941f24..01b502ffca792 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 @@ -41,7 +41,6 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; @@ -54,6 +53,7 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.SlotOwner; import org.apache.flink.runtime.operators.BatchTask; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; @@ -314,7 +314,7 @@ public void testAccumulatorsAndMetricsForwarding() throws Exception { Tuple2> graphAndExecutions = setupExecution(v1, 1, v2, 1); ExecutionGraph graph = graphAndExecutions.f0; - + // verify behavior for canceled executions Execution execution1 = graphAndExecutions.f1.values().iterator().next(); @@ -322,15 +322,15 @@ public void testAccumulatorsAndMetricsForwarding() throws Exception { Map> accumulators = new HashMap<>(); accumulators.put("acc", new IntCounter(4)); AccumulatorSnapshot accumulatorSnapshot = new AccumulatorSnapshot(graph.getJobID(), execution1.getAttemptId(), accumulators); - + TaskExecutionState state = new TaskExecutionState(graph.getJobID(), execution1.getAttemptId(), ExecutionState.CANCELED, null, accumulatorSnapshot, ioMetrics); - + graph.updateState(state); - + assertEquals(ioMetrics, execution1.getIOMetrics()); assertNotNull(execution1.getUserAccumulators()); assertEquals(4, execution1.getUserAccumulators().get("acc").getLocalValue()); - + // verify behavior for failed executions Execution execution2 = graphAndExecutions.f1.values().iterator().next(); @@ -361,14 +361,14 @@ public void testAccumulatorsAndMetricsStorage() throws Exception { JobVertex v2 = new JobVertex("v2", jid2); Map executions = setupExecution(v1, 1, v2, 1).f1; - + IOMetrics ioMetrics = new IOMetrics(0, 0, 0, 0, 0, 0.0, 0.0, 0.0, 0.0, 0.0); Map> accumulators = Collections.emptyMap(); Execution execution1 = executions.values().iterator().next(); execution1.cancel(); execution1.cancelingComplete(accumulators, ioMetrics); - + assertEquals(ioMetrics, execution1.getIOMetrics()); assertEquals(accumulators, execution1.getUserAccumulators()); @@ -534,7 +534,6 @@ private Tuple2> setupExecutio blobWriter, AkkaUtils.getDefaultTimeout()); checkJobOffloaded(eg); - eg.setQueuedSchedulingAllowed(false); List ordered = Arrays.asList(v1, v2); @@ -665,19 +664,6 @@ private SimpleSlot createSlot(TaskManagerLocation taskManagerLocation, int index new SimpleAckingTaskManagerGateway()); } - @SuppressWarnings("serial") - public static class FailingFinalizeJobVertex extends JobVertex { - - public FailingFinalizeJobVertex(String name, JobVertexID id) { - super(name, id); - } - - @Override - public void finalizeOnMaster(ClassLoader cl) throws Exception { - throw new Exception(); - } - } - private ExecutionGraph createExecutionGraph(Configuration configuration) throws Exception { final ScheduledExecutorService executor = TestingUtils.defaultExecutor(); From 736ea06eae96ad343e428c8b1178ffe119cb47df Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 27 Mar 2019 09:09:13 +0100 Subject: [PATCH 356/359] [FLINK-12021] Deploy execution in topological sorted order Due to changes how the slot futures are completed and due to the fact that the ResultConjunctFuture does not maintain the order in which the futures were specified, it could happen that executions were not deployed in topological order. This commit fixes this problem by changing the ResultConjunctFuture so that it maintains the order of the specified futures in its result collection. This closes #8065. --- .../flink/runtime/concurrent/FutureUtils.java | 12 +- .../concurrent/ConjunctFutureTest.java | 40 +++-- .../ExecutionGraphDeploymentTest.java | 158 ++++++++++++++++++ .../executiongraph/TestingSlotProvider.java | 2 +- .../taskmanager/LocalTaskManagerLocation.java | 2 +- 5 files changed, 193 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index 0f36a3af141db..414cb471d7003 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -518,9 +518,6 @@ private static class ResultConjunctFuture extends ConjunctFuture extends ConjunctFuture future : resultFutures) { - future.whenComplete(this::handleCompletedFuture); + final int index = counter; + counter++; + future.whenComplete((value, throwable) -> handleCompletedFuture(index, value, throwable)); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java index 9f33866bf1049..28fa8fb4c1e14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java @@ -18,21 +18,27 @@ package org.apache.flink.runtime.concurrent; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture; import org.apache.flink.util.TestLogger; -import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -40,7 +46,7 @@ import static org.junit.Assert.fail; /** - * Tests for the {@link ConjunctFuture} and {@link FutureUtils.WaitingConjunctFuture}. + * Tests for the {@link ConjunctFuture} and its sub classes. */ @RunWith(Parameterized.class) public class ConjunctFutureTest extends TestLogger { @@ -193,23 +199,33 @@ public void testConjunctFutureFailureOnSuccessive() throws Exception { } /** - * Tests that the conjunct future returns upon completion the collection of all future values. + * Tests that the conjunct future returns upon completion the collection of all future values + * in the same order in which the futures were inserted. */ @Test - public void testConjunctFutureValue() throws ExecutionException, InterruptedException { - java.util.concurrent.CompletableFuture future1 = java.util.concurrent.CompletableFuture.completedFuture(1); - java.util.concurrent.CompletableFuture future2 = java.util.concurrent.CompletableFuture.completedFuture(2L); - java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); + public void testConjunctFutureValue() throws Exception { + final int numberFutures = 10; - ConjunctFuture> result = FutureUtils.combineAll(Arrays.asList(future1, future2, future3)); + final List> futures = new ArrayList<>(numberFutures); + for (int i = 0; i < numberFutures; i++) { + futures.add(new CompletableFuture<>()); + } - assertFalse(result.isDone()); + ConjunctFuture> result = FutureUtils.combineAll(futures); - future3.complete(.1); + final List>> shuffledFutures = IntStream.range(0, futures.size()) + .mapToObj(index -> Tuple2.of(index, futures.get(index))) + .collect(Collectors.toList()); + Collections.shuffle(shuffledFutures); - assertTrue(result.isDone()); + for (Tuple2> shuffledFuture : shuffledFutures) { + assertThat(result.isDone(), is(false)); + shuffledFuture.f1.complete(shuffledFuture.f0); + } + + assertThat(result.isDone(), is(true)); - assertThat(result.get(), IsIterableContainingInAnyOrder.containsInAnyOrder(1, 2L, .1)); + assertThat(result.get(), is(equalTo(IntStream.range(0, numberFutures).boxed().collect(Collectors.toList())))); } @Test 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 01b502ffca792..cb9126055c9a3 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 @@ -33,6 +33,8 @@ import org.apache.flink.runtime.blob.VoidBlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; @@ -42,6 +44,7 @@ import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.SlotSharingGroupId; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -53,20 +56,32 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; import org.apache.flink.runtime.jobmaster.SlotOwner; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlot; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.operators.BatchTask; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; import org.junit.Test; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -74,9 +89,11 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.function.Function; import static junit.framework.TestCase.assertTrue; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance; @@ -84,6 +101,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -656,6 +674,112 @@ public void testEagerSchedulingWaitsOnAllInputPreferredLocations() throws Except } } + /** + * Tests that the {@link ExecutionGraph} is deployed in topological order. + */ + @Test + public void testExecutionGraphIsDeployedInTopologicalOrder() throws Exception { + final int sourceParallelism = 2; + final int sinkParallelism = 1; + + final JobVertex sourceVertex = new JobVertex("source"); + sourceVertex.setInvokableClass(NoOpInvokable.class); + sourceVertex.setParallelism(sourceParallelism); + + final JobVertex sinkVertex = new JobVertex("sink"); + sinkVertex.setInvokableClass(NoOpInvokable.class); + sinkVertex.setParallelism(sinkParallelism); + + sinkVertex.connectNewDataSetAsInput(sourceVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); + + final JobID jobId = new JobID(); + final int numberTasks = sourceParallelism + sinkParallelism; + final ArrayBlockingQueue submittedTasksQueue = new ArrayBlockingQueue<>(numberTasks); + TestingTaskExecutorGatewayBuilder testingTaskExecutorGatewayBuilder = new TestingTaskExecutorGatewayBuilder(); + testingTaskExecutorGatewayBuilder.setSubmitTaskConsumer((taskDeploymentDescriptor, jobMasterId) -> { + submittedTasksQueue.offer(taskDeploymentDescriptor.getExecutionAttemptId()); + return CompletableFuture.completedFuture(Acknowledge.get()); + }); + + final TestingTaskExecutorGateway taskExecutorGateway = testingTaskExecutorGatewayBuilder.createTestingTaskExecutorGateway(); + final RpcTaskManagerGateway taskManagerGateway = new RpcTaskManagerGateway(taskExecutorGateway, JobMasterId.generate()); + + final Collection> slotFutures = new ArrayList<>(numberTasks); + for (int i = 0; i < numberTasks; i++) { + slotFutures.add(new CompletableFuture<>()); + } + + final SlotProvider slotProvider = new IteratorTestingSlotProvider(slotFutures.iterator()); + + final ExecutionGraph executionGraph = ExecutionGraphTestUtils.createExecutionGraph( + jobId, + slotProvider, + new NoRestartStrategy(), + new DirectScheduledExecutorService(), + sourceVertex, + sinkVertex); + executionGraph.setScheduleMode(ScheduleMode.EAGER); + executionGraph.setQueuedSchedulingAllowed(true); + + executionGraph.scheduleForExecution(); + + // change the order in which the futures are completed + final List> shuffledFutures = new ArrayList<>(slotFutures); + Collections.shuffle(shuffledFutures); + + for (CompletableFuture slotFuture : shuffledFutures) { + slotFuture.complete(new TestingLogicalSlot( + new LocalTaskManagerLocation(), + taskManagerGateway, + 0, + new AllocationID(), + new SlotRequestId(), + new SlotSharingGroupId(), + null)); + } + + final List submittedTasks = new ArrayList<>(numberTasks); + + for (int i = 0; i < numberTasks; i++) { + submittedTasks.add(submittedTasksQueue.take()); + } + + final Collection firstStage = new ArrayList<>(sourceParallelism); + for (ExecutionVertex taskVertex : executionGraph.getJobVertex(sourceVertex.getID()).getTaskVertices()) { + firstStage.add(taskVertex.getCurrentExecutionAttempt().getAttemptId()); + } + + final Collection secondStage = new ArrayList<>(sinkParallelism); + for (ExecutionVertex taskVertex : executionGraph.getJobVertex(sinkVertex.getID()).getTaskVertices()) { + secondStage.add(taskVertex.getCurrentExecutionAttempt().getAttemptId()); + } + + assertThat(submittedTasks, new ExecutionStageMatcher(Arrays.asList(firstStage, secondStage))); + } + + private static final class IteratorTestingSlotProvider extends TestingSlotProvider { + private IteratorTestingSlotProvider(final Iterator> slotIterator) { + super(new IteratorSlotFutureFunction(slotIterator)); + } + + private static class IteratorSlotFutureFunction implements Function> { + final Iterator> slotIterator; + + IteratorSlotFutureFunction(Iterator> slotIterator) { + this.slotIterator = slotIterator; + } + + @Override + public CompletableFuture apply(SlotRequestId slotRequestId) { + if (slotIterator.hasNext()) { + return slotIterator.next(); + } else { + return FutureUtils.completedExceptionally(new FlinkException("No more slots available.")); + } + } + } + } + private SimpleSlot createSlot(TaskManagerLocation taskManagerLocation, int index) { return new SimpleSlot( mock(SlotOwner.class), @@ -701,4 +825,38 @@ private ExecutionGraph createExecutionGraph(Configuration configuration) throws timeout, LoggerFactory.getLogger(getClass())); } + + private static final class ExecutionStageMatcher extends TypeSafeMatcher> { + private final List> executionStages; + + private ExecutionStageMatcher(List> executionStages) { + this.executionStages = executionStages; + } + + @Override + protected boolean matchesSafely(List submissionOrder) { + final Iterator submissionIterator = submissionOrder.iterator(); + + for (Collection stage : executionStages) { + final Collection currentStage = new ArrayList<>(stage); + + while (!currentStage.isEmpty() && submissionIterator.hasNext()) { + if (!currentStage.remove(submissionIterator.next())) { + return false; + } + } + + if (!currentStage.isEmpty()) { + return false; + } + } + + return !submissionIterator.hasNext(); + } + + @Override + public void describeTo(Description description) { + description.appendValueList("<[", ", ", "]>", executionStages); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java index ed8fe130de6be..312ab110b6cc3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java @@ -39,7 +39,7 @@ /** * {@link SlotProvider} implementation for testing purposes. */ -final class TestingSlotProvider implements SlotProvider { +public class TestingSlotProvider implements SlotProvider { private final ConcurrentMap> slotFutures; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalTaskManagerLocation.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalTaskManagerLocation.java index 60dddbb8b120c..1c46891e9e253 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalTaskManagerLocation.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalTaskManagerLocation.java @@ -30,6 +30,6 @@ public class LocalTaskManagerLocation extends TaskManagerLocation { private static final long serialVersionUID = 2396142513336559461L; public LocalTaskManagerLocation() { - super(ResourceID.generate(), InetAddress.getLoopbackAddress(), -1); + super(ResourceID.generate(), InetAddress.getLoopbackAddress(), 42); } } From 4b784b256b7884fa6f9070fab4e7ca1e36587d1c Mon Sep 17 00:00:00 2001 From: jackyyin Date: Sun, 31 Mar 2019 15:31:13 +0800 Subject: [PATCH 357/359] [FLINK-12020] [docs] Add documentation for mesos-appmaster-job.sh 1. Rename the section title from `standalone` to `Flink session cluster on Mesos`. 2. Add one new section `Flink job cluster on Mesos` which describes the usage of mesos-appmaster-job.sh. This closes #8084. --- docs/ops/deployment/mesos.md | 35 ++++++++++++++++++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md index b36bdec896b35..8395bd0750662 100644 --- a/docs/ops/deployment/mesos.md +++ b/docs/ops/deployment/mesos.md @@ -154,7 +154,9 @@ If set to 'docker', specify the image name: mesos.resourcemanager.tasks.container.image.name: image_name -### Standalone +### Flink session cluster on Mesos + +A Flink session cluster is executed as a long-running Mesos Deployment. Note that you can run multiple Flink jobs on a session cluster. Each job needs to be submitted to the cluster after the cluster has been deployed. In the `/bin` directory of the Flink distribution, you find two startup scripts which manage the Flink processes in a Mesos cluster: @@ -173,6 +175,37 @@ In order to run the `mesos-appmaster.sh` script you have to define `mesos.master When executing `mesos-appmaster.sh`, it will create a job manager on the machine where you executed the script. In contrast to that, the task managers will be run as Mesos tasks in the Mesos cluster. +### Flink job cluster on Mesos + +A Flink job cluster is a dedicated cluster which runs a single job. +There is no extra job submission needed. + +In the `/bin` directory of the Flink distribution, you find one startup script +which manage the Flink processes in a Mesos cluster: + +1. `mesos-appmaster-job.sh` + This starts the Mesos application master which will register the Mesos scheduler, retrieve the job graph and then launch the task managers accordingly. + +In order to run the `mesos-appmaster-job.sh` script you have to define `mesos.master` and `internal.jobgraph-path` in the `flink-conf.yaml` +or pass it via `-Dmesos.master=... -Dinterval.jobgraph-path=...` to the Java process. + +The job graph file may be generated like this way: + +{% highlight java %} +final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); +jobGraph.setAllowQueuedScheduling(true); +final String jobGraphFilename = "job.graph"; +File jobGraphFile = new File(jobGraphFilename); +try (FileOutputStream output = new FileOutputStream(jobGraphFile); + ObjectOutputStream obOutput = new ObjectOutputStream(output)){ + obOutput.writeObject(jobGraph); +} +{% endhighlight %} + +Note: +1. Before serializing the job graph, please make sure to enable queued scheduling because slots need to be allocated lazily +2. Make sure that all Mesos processes have the user code jar on the classpath (e.g. putting them in the lib directory) + #### General configuration It is possible to completely parameterize a Mesos application through Java properties passed to the Mesos application master. From b75463a2456f40df51b0cc5eb2ff712642b2629e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 1 Apr 2019 12:39:12 -0700 Subject: [PATCH 358/359] [FLINK-12075] Update flink-conf.yaml to not specify rest.port per default This commits updates the flink-conf.yaml to contain the new rest options and comments out the rest.port per default. --- flink-dist/src/main/resources/flink-conf.yaml | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index efdf33e3852a9..99ea6851165f3 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -121,17 +121,20 @@ parallelism.default: 1 # state.backend.incremental: false #============================================================================== -# Web Frontend +# Rest & web frontend #============================================================================== -# The address under which the web-based runtime monitor listens. +# The port to which the REST client connects to and the server binds to. # -#web.address: 0.0.0.0 +#rest.port: 8081 -# The port under which the web-based runtime monitor listens. -# A value of -1 deactivates the web server. +# The address to which the REST client will connect to +# +#rest.address: 0.0.0.0 -rest.port: 8081 +# The address that the REST & web server binds to +# +#rest.bind-address: 0.0.0.0 # Flag to specify whether job submission is enabled from the web-based # runtime monitor. Uncomment to disable. From 440f8b7b5f6ea334f466bbfbf9569e9a67391d28 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 1 Apr 2019 13:02:57 -0700 Subject: [PATCH 359/359] [FLINK-12075][yarn] Set RestOptions.BIND_PORT only to 0 if not specified This commit changes the YarnEntrypointUtils#loadConfiguration so that it only sets RestOptions.BIND_PORT to 0 if it has not been specified. This allows to explicitly set a port range for Yarn applications which are running behind a firewall, for example. --- .../yarn/entrypoint/YarnEntrypointUtils.java | 2 +- .../entrypoint/YarnEntrypointUtilsTest.java | 93 +++++++++++++++++++ 2 files changed, 94 insertions(+), 1 deletion(-) create mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java index 5566963cb3592..af7bffaae2c5b 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java @@ -98,7 +98,7 @@ public static Configuration loadConfiguration(String workingDirectory, Map= 0) { + if (!configuration.contains(RestOptions.PORT)) { // set the REST port to 0 to select it randomly configuration.setInteger(RestOptions.PORT, 0); } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java new file mode 100644 index 0000000000000..c06e270f19164 --- /dev/null +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.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.yarn.entrypoint; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.util.TestLogger; + +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link YarnEntrypointUtils}. + */ +public class YarnEntrypointUtilsTest extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(YarnEntrypointUtilsTest.class); + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + /** + * Tests that the REST ports are correctly set when loading a {@link Configuration} + * with unspecified REST options. + */ + @Test + public void testRestPortOptionsUnspecified() throws IOException { + final Configuration initialConfiguration = new Configuration(); + + final Configuration configuration = loadConfiguration(initialConfiguration); + + // having not specified the ports should set the rest bind port to 0 + assertThat(configuration.getInteger(RestOptions.PORT), is(equalTo(0))); + } + + /** + * Tests that the REST port option is respected if it has been specified. + */ + @Test + public void testRestPortSpecified() throws IOException { + final Configuration initialConfiguration = new Configuration(); + final int port = 1337; + initialConfiguration.setInteger(RestOptions.PORT, port); + + final Configuration configuration = loadConfiguration(initialConfiguration); + + // if the bind port is not specified it should fall back to the rest port + assertThat(configuration.getInteger(RestOptions.PORT), is(equalTo(port))); + } + + @Nonnull + private Configuration loadConfiguration(Configuration initialConfiguration) throws IOException { + final File workingDirectory = TEMPORARY_FOLDER.newFolder(); + final Map env = new HashMap<>(4); + env.put(ApplicationConstants.Environment.NM_HOST.key(), "foobar"); + + BootstrapTools.writeConfiguration(initialConfiguration, new File(workingDirectory, "flink-conf.yaml")); + return YarnEntrypointUtils.loadConfiguration(workingDirectory.getAbsolutePath(), env, LOG); + } + +}